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. --- examples/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'examples') 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 -- 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 'examples') 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 b7794813b181f13801596e8d8c3b4471c0c84f20 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 15 Jun 2013 09:37:13 -0700 Subject: Fix run script on Windows for Scala 2.10 --- examples/src/main/scala/spark/examples/SparkPi.scala | 3 ++- run2.cmd | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) (limited to 'examples') diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index f598d2ff9c..b1c736a2cd 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -21,6 +21,7 @@ object SparkPi { if (x*x + y*y < 1) 1 else 0 }.reduce(_ + _) println("Pi is roughly " + 4.0 * count / n) - System.exit(0) + //System.exit(0) + spark.stop() } } diff --git a/run2.cmd b/run2.cmd index d2d4807971..27d0968571 100644 --- a/run2.cmd +++ b/run2.cmd @@ -1,6 +1,6 @@ @echo off -set SCALA_VERSION=2.9.3 +set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed set FWDIR=%~dp0 -- 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 'examples') 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 'examples') 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 'examples') 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 'examples') 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 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 'examples') 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 'examples') 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 6e8a96c7e7652b3d1fc709b3c3ccc5f90ffeb623 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 13 Dec 2013 23:14:08 -0800 Subject: Fix maven build issues in 2.10 branch --- examples/pom.xml | 2 +- pom.xml | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) (limited to 'examples') diff --git a/examples/pom.xml b/examples/pom.xml index 97f6dfea66..e16b1c80aa 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -87,7 +87,7 @@ - org.apache.kafka + com.sksamuel.kafka kafka_2.10 0.8.0-beta1 diff --git a/pom.xml b/pom.xml index aee07c2138..58fc02e029 100644 --- a/pom.xml +++ b/pom.xml @@ -498,6 +498,10 @@ org.jboss.netty netty + + io.netty + netty + -- 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 'examples') 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 From a8bb86389d8dc8efeff83561aea044a3c4924df5 Mon Sep 17 00:00:00 2001 From: azuryyu Date: Tue, 24 Dec 2013 16:52:20 +0800 Subject: Fixed job name in the java streaming example. --- .../java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'examples') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 9a8e4209ed..22994fb2ec 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -53,7 +53,7 @@ public class JavaKafkaWordCount { } // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); int numThreads = Integer.parseInt(args[4]); -- cgit v1.2.3 From 2573add94cf920a88f74d80d8ea94218d812704d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 24 Dec 2013 18:30:31 +0530 Subject: spark-544, introducing SparkConf and related configuration overhaul. --- .../scala/org/apache/spark/MapOutputTracker.scala | 7 +- .../main/scala/org/apache/spark/Partitioner.scala | 4 +- .../main/scala/org/apache/spark/SparkConf.scala | 71 +++++++++++ .../main/scala/org/apache/spark/SparkContext.scala | 140 +++++++++++---------- .../src/main/scala/org/apache/spark/SparkEnv.scala | 44 ++++--- .../org/apache/spark/api/python/PythonRDD.scala | 6 +- .../org/apache/spark/broadcast/Broadcast.scala | 6 +- .../apache/spark/broadcast/BroadcastFactory.scala | 4 +- .../org/apache/spark/broadcast/HttpBroadcast.scala | 14 +-- .../apache/spark/broadcast/TorrentBroadcast.scala | 9 +- .../spark/deploy/ApplicationDescription.scala | 2 +- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 3 +- .../org/apache/spark/deploy/client/Client.scala | 13 +- .../apache/spark/deploy/client/TestClient.scala | 7 +- .../org/apache/spark/deploy/master/Master.scala | 31 ++--- .../spark/deploy/master/MasterArguments.scala | 7 +- .../deploy/master/SparkZooKeeperSession.scala | 7 +- .../master/ZooKeeperLeaderElectionAgent.scala | 9 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 8 +- .../spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../org/apache/spark/deploy/worker/Worker.scala | 28 ++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 6 +- .../executor/CoarseGrainedExecutorBackend.scala | 6 +- .../scala/org/apache/spark/executor/Executor.scala | 10 +- .../org/apache/spark/io/CompressionCodec.scala | 14 ++- .../org/apache/spark/metrics/MetricsSystem.scala | 10 +- .../apache/spark/network/ConnectionManager.scala | 24 ++-- .../org/apache/spark/network/ReceiverTest.scala | 4 +- .../org/apache/spark/network/SenderTest.scala | 4 +- .../apache/spark/network/netty/ShuffleCopier.scala | 8 +- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 6 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + .../org/apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/scheduler/SchedulableBuilder.scala | 6 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 11 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 19 +-- .../cluster/CoarseGrainedSchedulerBackend.scala | 12 +- .../scheduler/cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +- .../spark/scheduler/cluster/TaskResultGetter.scala | 3 +- .../mesos/CoarseMesosSchedulerBackend.scala | 10 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalScheduler.scala | 5 +- .../apache/spark/serializer/KryoSerializer.scala | 10 +- .../spark/storage/BlockFetcherIterator.scala | 4 +- .../org/apache/spark/storage/BlockManager.scala | 38 +++--- .../apache/spark/storage/BlockManagerMaster.scala | 11 +- .../spark/storage/BlockManagerMasterActor.scala | 10 +- .../apache/spark/storage/BlockObjectWriter.scala | 5 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockManager.scala | 7 +- .../org/apache/spark/storage/ThreadingTest.scala | 6 +- .../main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 4 +- .../org/apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 25 ++-- .../org/apache/spark/util/MetadataCleaner.scala | 12 +- .../org/apache/spark/util/SizeEstimator.scala | 7 +- .../main/scala/org/apache/spark/util/Utils.scala | 7 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 16 +-- .../apache/spark/metrics/MetricsSystemSuite.scala | 8 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 23 ++-- .../apache/spark/scheduler/JobLoggerSuite.scala | 2 +- .../scheduler/cluster/ClusterSchedulerSuite.scala | 2 +- .../cluster/ClusterTaskSetManagerSuite.scala | 4 +- .../scheduler/cluster/TaskResultGetterSuite.scala | 2 +- .../apache/spark/storage/BlockManagerSuite.scala | 95 +++++++------- .../spark/storage/DiskBlockManagerSuite.scala | 12 +- .../org/apache/spark/util/SizeEstimatorSuite.scala | 4 +- .../spark/examples/bagel/WikipediaPageRank.scala | 10 +- .../bagel/WikipediaPageRankStandalone.scala | 8 +- .../spark/streaming/examples/ActorWordCount.scala | 3 +- .../apache/spark/mllib/recommendation/ALS.scala | 13 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +-- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 4 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- project/SparkBuild.scala | 3 +- .../scala/org/apache/spark/repl/SparkILoop.scala | 7 +- .../scala/org/apache/spark/repl/SparkIMain.scala | 7 +- .../org/apache/spark/streaming/Checkpoint.scala | 3 +- .../org/apache/spark/streaming/Scheduler.scala | 6 +- .../apache/spark/streaming/StreamingContext.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 6 +- .../apache/spark/streaming/CheckpointSuite.scala | 6 +- .../apache/spark/streaming/InputStreamsSuite.scala | 18 +-- .../org/apache/spark/streaming/TestSuiteBase.scala | 11 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +-- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- 96 files changed, 612 insertions(+), 478 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SparkConf.scala (limited to 'examples') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ccffcc356c..4520edb10d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -50,9 +50,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker extends Logging { +private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - private val timeout = AkkaUtils.askTimeout + private val timeout = AkkaUtils.askTimeout(conf) // Set to the MapOutputTrackerActor living on the driver var trackerActor: Either[ActorRef, ActorSelection] = _ @@ -192,7 +192,8 @@ private[spark] class MapOutputTracker extends Logging { } } -private[spark] class MapOutputTrackerMaster extends MapOutputTracker { +private[spark] class MapOutputTrackerMaster(conf: SparkConf) + extends MapOutputTracker(conf) { // Cache a serialized version of the output statuses for each shuffle to send them out faster private var cacheEpoch = epoch diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index bcec41c439..04c1eedfeb 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -32,6 +32,8 @@ abstract class Partitioner extends Serializable { } object Partitioner { + + import SparkContext.{globalConf => conf} /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -52,7 +54,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (System.getProperty("spark.default.parallelism") != null) { + if (conf.getOrElse("spark.default.parallelism", null) != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala new file mode 100644 index 0000000000..9a4eefad2e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -0,0 +1,71 @@ +package org.apache.spark + +import scala.collection.JavaConversions._ +import scala.collection.concurrent.TrieMap + +import com.typesafe.config.ConfigFactory + +private[spark] class SparkConf(loadClasspathRes: Boolean = true) extends Serializable { + @transient lazy val config = ConfigFactory.systemProperties() + .withFallback(ConfigFactory.parseResources("spark.conf")) + // TODO this should actually be synchronized + private val configMap = TrieMap[String, String]() + + if (loadClasspathRes && !config.entrySet().isEmpty) { + for (e <- config.entrySet()) { + configMap += ((e.getKey, e.getValue.unwrapped().toString)) + } + } + + def setMasterUrl(master: String) = { + if (master != null) + configMap += (("spark.master", master)) + this + } + + def setAppName(name: String) = { + if (name != null) + configMap += (("spark.appName", name)) + this + } + + def setJars(jars: Seq[String]) = { + if (!jars.isEmpty) + configMap += (("spark.jars", jars.mkString(","))) + this + } + + def set(k: String, value: String) = { + configMap += ((k, value)) + this + } + + def setSparkHome(home: String) = { + if (home != null) + configMap += (("spark.home", home)) + this + } + + def set(map: Seq[(String, String)]) = { + if (map != null && !map.isEmpty) + configMap ++= map + this + } + + def get(k: String): String = { + configMap(k) + } + + def getAllConfiguration = configMap.clone.entrySet().iterator + + def getOrElse(k: String, defaultValue: String): String = { + configMap.getOrElse(k, defaultValue) + } + + override def clone: SparkConf = { + val conf = new SparkConf(false) + conf.set(configMap.toSeq) + conf + } + +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a0f794edfd..4300b07bdb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,91 +22,99 @@ import java.net.URI import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.Map +import scala.collection.{Map, immutable} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap + +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.ArrayWritable -import org.apache.hadoop.io.BooleanWritable -import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.DoubleWritable -import org.apache.hadoop.io.FloatWritable -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.FileInputFormat -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.SequenceFileInputFormat -import org.apache.hadoop.mapred.TextInputFormat -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, +FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, +TextInputFormat} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} - import org.apache.mesos.MesosNativeLibrary import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend, +SimrSchedulerBackend, SparkDeploySchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, +MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalScheduler -import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, - TimeStampedHashMap, Utils} +import org.apache.spark.util._ /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param appName A name for your application, to display on the cluster web UI. - * @param sparkHome Location where Spark is installed on cluster nodes. - * @param jars Collection of JARs to send to the cluster. These can be paths on the local file - * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param conf a Spark Config object describing the context configuration. Any settings in this + * config overrides the default configs as well as system properties. + * * @param environment Environment variables to set on worker nodes. */ class SparkContext( - val master: String, - val appName: String, - val sparkHome: String = null, - val jars: Seq[String] = Nil, + val conf: SparkConf, val environment: Map[String, String] = Map(), // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - scala.collection.immutable.Map()) + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = immutable.Map()) extends Logging { - // Ensure logging is initialized before we spawn any threads - initLogging() + /** + * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark + * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param environment Environment variables to set on worker nodes. + */ + def this(master: String, appName: String, sparkHome: String = null, + jars: Seq[String] = Nil, environment: Map[String, String] = Map(), + preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = + immutable.Map()) = + this(new SparkConf(false).setAppName(appName).setMasterUrl(master) + .setJars(jars).set(environment.toSeq).setSparkHome(sparkHome), + environment, preferredNodeLocationData) // Set Spark driver host and port system properties - if (System.getProperty("spark.driver.host") == null) { - System.setProperty("spark.driver.host", Utils.localHostName()) - } - if (System.getProperty("spark.driver.port") == null) { - System.setProperty("spark.driver.port", "0") - } + Try(conf.get("spark.driver.host")) + .getOrElse(conf.set("spark.driver.host", Utils.localHostName())) + + Try(conf.get("spark.driver.port")) + .getOrElse(conf.set("spark.driver.port", "0")) + + val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) { + conf.get("spark.jars").split(",") + } else null + + val master = conf.get("spark.master") + val appName = conf.get("spark.appName") val isLocal = (master == "local" || master.startsWith("local[")) + // Ensure logging is initialized before we spawn any threads + initLogging() + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( "", - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port").toInt, + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + conf, true, isLocal) SparkEnv.set(env) @@ -165,24 +173,24 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { val env = SparkEnv.get - val conf = SparkHadoopUtil.get.newConfiguration() + val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) } // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" Utils.getSystemProperties.foreach { case (key, value) => if (key.startsWith("spark.hadoop.")) { - conf.set(key.substring("spark.hadoop.".length), value) + hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - conf.set("io.file.buffer.size", bufferSize) - conf + val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf } private[spark] var checkpointDir: Option[String] = None @@ -695,10 +703,8 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (sparkHome != null) { - Some(sparkHome) - } else if (System.getProperty("spark.home") != null) { - Some(System.getProperty("spark.home")) + if (conf.getOrElse("spark.home", null) != null) { + Some(conf.get("spark.home")) } else if (System.getenv("SPARK_HOME") != null) { Some(System.getenv("SPARK_HOME")) } else { @@ -909,6 +915,14 @@ object SparkContext { private[spark] val SPARK_UNKNOWN_USER = "" + private lazy val conf = new SparkConf() + + private[spark] def globalConf = { + if (SparkEnv.get != null) { + SparkEnv.get.conf + } else conf + } + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 @@ -1020,7 +1034,7 @@ object SparkContext { /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ private[spark] val executorMemoryRequested = { // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) + Try(globalConf.get("spark.executor.memory")).toOption .orElse(Option(System.getenv("SPARK_MEM"))) .map(Utils.memoryStringToMb) .getOrElse(512) @@ -1123,7 +1137,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(sc) - val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean + val coarseGrained = globalConf.getOrElse("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 826f5c2d8c..78e4ae27b2 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -54,7 +54,8 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem) { + val metricsSystem: MetricsSystem, + val conf: SparkConf) { private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -114,25 +115,27 @@ object SparkEnv extends Logging { executorId: String, hostname: String, port: Int, + conf: SparkConf, isDriver: Boolean, isLocal: Boolean): SparkEnv = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, + conf = conf) // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), // figure out which port number Akka actually bound to and set spark.driver.port to it. if (isDriver && port == 0) { - System.setProperty("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", boundPort.toString) } // set only if unset until now. - if (System.getProperty("spark.hostPort", null) == null) { + if (conf.getOrElse("spark.hostPort", null) == null) { if (!isDriver){ // unexpected Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set") } Utils.checkHost(hostname) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + conf.set("spark.hostPort", hostname + ":" + boundPort) } val classLoader = Thread.currentThread.getContextClassLoader @@ -140,25 +143,25 @@ object SparkEnv extends Logging { // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = System.getProperty(propertyName, defaultClassName) + val name = conf.getOrElse(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + 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 + val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") + val driverPort: Int = conf.getOrElse("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) @@ -168,21 +171,21 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) + new BlockManagerMasterActor(isLocal, conf)), conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) val connectionManager = blockManager.connectionManager - val broadcastManager = new BroadcastManager(isDriver) + val broadcastManager = new BroadcastManager(isDriver, conf) val cacheManager = new CacheManager(blockManager) // Have to assign trackerActor after initialization as MapOutputTrackerActor // requires the MapOutputTracker itself val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster() + new MapOutputTrackerMaster(conf) } else { - new MapOutputTracker() + new MapOutputTracker(conf) } mapOutputTracker.trackerActor = registerOrLookup( "MapOutputTracker", @@ -193,12 +196,12 @@ object SparkEnv extends Logging { val httpFileServer = new HttpFileServer() httpFileServer.initialize() - System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + conf.set("spark.fileserver.uri", httpFileServer.serverUri) val metricsSystem = if (isDriver) { - MetricsSystem.createMetricsSystem("driver") + MetricsSystem.createMetricsSystem("driver", conf) } else { - MetricsSystem.createMetricsSystem("executor") + MetricsSystem.createMetricsSystem("executor", conf) } metricsSystem.start() @@ -212,7 +215,7 @@ object SparkEnv extends Logging { } // Warn about deprecated spark.cache.class property - if (System.getProperty("spark.cache.class") != null) { + if (conf.getOrElse("spark.cache.class", null) != null) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + "levels using the RDD.persist() method instead.") } @@ -231,6 +234,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem) + metricsSystem, + conf) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ca42c76928..d6eacfe23e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -247,10 +247,10 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[ */ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - + import SparkContext.{globalConf => conf} Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 43c18294c5..be99d229ef 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -32,7 +32,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { } private[spark] -class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable { +class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { private var initialized = false private var broadcastFactory: BroadcastFactory = null @@ -43,14 +43,14 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable private def initialize() { synchronized { if (!initialized) { - val broadcastFactoryClass = System.getProperty( + val broadcastFactoryClass = conf.getOrElse( "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver) + broadcastFactory.initialize(isDriver, conf) initialized = true } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 68bff75b90..fb161ce69d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import org.apache.spark.SparkConf + /** * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified @@ -24,7 +26,7 @@ package org.apache.spark.broadcast * entire Spark job. */ private[spark] trait BroadcastFactory { - def initialize(isDriver: Boolean): Unit + def initialize(isDriver: Boolean, conf: SparkConf): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 47db720416..cecb8c228b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkEnv} +import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} @@ -64,7 +64,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } private[spark] class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) @@ -88,15 +88,16 @@ private object HttpBroadcast extends Logging { private lazy val compressionCodec = CompressionCodec.createCodec() - def initialize(isDriver: Boolean) { + def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - compress = System.getProperty("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean if (isDriver) { createServer() + conf.set("spark.httpBroadcast.uri", serverUri) } - serverUri = System.getProperty("spark.httpBroadcast.uri") + serverUri = conf.get("spark.httpBroadcast.uri") initialized = true } } @@ -118,7 +119,6 @@ private object HttpBroadcast extends Logging { server = new HttpServer(broadcastDir) server.start() serverUri = server.uri - System.setProperty("spark.httpBroadcast.uri", serverUri) logInfo("Broadcast server started at " + serverUri) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 073a0a5029..4a3801dc48 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -166,8 +166,9 @@ private object TorrentBroadcast extends Logging { private var initialized = false - - def initialize(_isDriver: Boolean) { + private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { + TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests synchronized { if (!initialized) { initialized = true @@ -179,7 +180,7 @@ extends Logging { initialized = false } - val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) @@ -238,7 +239,7 @@ private[spark] case class TorrentInfo( private[spark] class TorrentBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 19d393a0db..dda43dc018 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -26,7 +26,7 @@ private[spark] class ApplicationDescription( val appUiUrl: String) extends Serializable { - val user = System.getProperty("user.name", "") + val user = System.getProperty("user.name", "") override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index fc1537f796..1c979ac3e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -67,8 +67,9 @@ class SparkHadoopUtil { } object SparkHadoopUtil { + import SparkContext.{globalConf => conf} private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if (yarnMode) { try { Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] 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 953755e40d..9bbd635ab9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -19,20 +19,18 @@ package org.apache.spark.deploy.client import java.util.concurrent.TimeoutException -import scala.concurrent.duration._ import scala.concurrent.Await +import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.util.AkkaUtils - /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, * and a listener for cluster events, and calls back the listener when various events occur. @@ -43,7 +41,8 @@ private[spark] class Client( actorSystem: ActorSystem, masterUrls: Array[String], appDescription: ApplicationDescription, - listener: ClientListener) + listener: ClientListener, + conf: SparkConf) extends Logging { val REGISTRATION_TIMEOUT = 20.seconds @@ -178,7 +177,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 5b62d3ba6c..426cf524ae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{Logging} +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { @@ -45,11 +45,12 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) + val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + conf = SparkContext.globalConf) val desc = new ApplicationDescription( "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener - val client = new Client(actorSystem, Array(url), desc, listener) + val client = new Client(actorSystem, Array(url), desc, listener, SparkContext.globalConf) client.start() actorSystem.awaitTermination() } 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 eebd0794b8..2c162c4fa2 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 @@ -29,7 +29,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkContext, Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ @@ -39,13 +39,13 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher - + val conf = SparkContext.globalConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -63,8 +63,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications") + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf) + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf) val masterSource = new MasterSource(this) val webUi = new MasterWebUI(this, webUiPort) @@ -86,7 +86,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -103,7 +103,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act persistenceEngine = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - new ZooKeeperPersistenceEngine(SerializationExtension(context.system)) + new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf) case "FILESYSTEM" => logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system)) @@ -113,7 +113,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act leaderElectionAgent = RECOVERY_MODE match { case "ZOOKEEPER" => - context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl)) + context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf)) case _ => context.actorOf(Props(classOf[MonarchyLeaderAgent], self)) } @@ -507,7 +507,7 @@ private[spark] object Master { val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { - val args = new MasterArguments(argStrings) + val args = new MasterArguments(argStrings, SparkContext.globalConf) val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) actorSystem.awaitTermination() } @@ -523,9 +523,10 @@ private[spark] object Master { } def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = SparkContext.globalConf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(SparkContext.globalConf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 9d89b455fb..7ce83f9c36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -18,11 +18,12 @@ package org.apache.spark.deploy.master import org.apache.spark.util.{Utils, IntParam} +import org.apache.spark.SparkConf /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String]) { +private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -37,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String]) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (System.getProperty("master.ui.port") != null) { - webUiPort = System.getProperty("master.ui.port").toInt + if (conf.get("master.ui.port") != null) { + webUiPort = conf.get("master.ui.port").toInt } parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 6cc7fd2ff4..79d95b1a83 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.data.Stat -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry @@ -35,8 +35,9 @@ import org.apache.spark.Logging * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many * times or a semantic exception is thrown (e.g., "node already exists"). */ -private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { - val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "") +private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, + conf: SparkConf) extends Logging { + val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 7d535b08de..df5bb368a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,16 +21,17 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.master.MasterMessages._ -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) +private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, + masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() - private val zk = new SparkZooKeeperSession(this) + private val zk = new SparkZooKeeperSession(this, conf) private var status = LeadershipStatus.NOT_LEADER private var myLeaderFile: String = _ private var leaderUrl: String = _ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 825344b3bb..c55b720422 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,19 +17,19 @@ package org.apache.spark.deploy.master -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.zookeeper._ import akka.serialization.Serialization -class ZooKeeperPersistenceEngine(serialization: Serialization) +class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk = new SparkZooKeeperSession(this) + val zk = new SparkZooKeeperSession(this, conf) zk.connect() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 9ab594b682..ead35662fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(master.conf) val host = Utils.localHostName() val port = requestedPort 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 87531b6719..75a6e75c78 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,23 +25,14 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.WorkerStateResponse -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.util.{AkkaUtils, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -53,7 +44,8 @@ private[spark] class Worker( cores: Int, memory: Int, masterUrls: Array[String], - workDirPath: String = null) + workDirPath: String = null, + val conf: SparkConf) extends Actor with Logging { import context.dispatcher @@ -63,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -92,7 +84,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker") + val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf) val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -275,6 +267,7 @@ private[spark] class Worker( } private[spark] object Worker { + import org.apache.spark.SparkContext.globalConf def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, @@ -287,9 +280,10 @@ private[spark] object Worker { : (ActorSystem, Int) = { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = globalConf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir), name = "Worker") + masterUrls, workDir, globalConf), name = "Worker") (actorSystem, boundPort) } 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 40d6bdb3fd..ec47ba1b56 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 @@ -22,7 +22,7 @@ import java.io.File import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ @@ -34,10 +34,10 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( - System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getOrElse("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) var server: Option[Server] = None var boundPort: Option[Int] = None 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 debbdd4c44..c8319f6f6e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} @@ -98,10 +98,10 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true) + indestructible = true, conf = SparkContext.globalConf) // set it val sparkHostPort = hostname + ":" + boundPort - System.setProperty("spark.hostPort", sparkHostPort) +// conf.set("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 0f19d7a96b..70fc30e993 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -57,17 +57,17 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) - + val conf = new SparkConf(false) // Set spark.* system properties from executor arg for ((key, value) <- properties) { - System.setProperty(key, value) + conf.set(key, value) } // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) { - System.setProperty("spark.local.dir", getYarnLocalDirs()) + conf.set("spark.local.dir", getYarnLocalDirs()) } // Create our ClassLoader and set it on this thread @@ -108,7 +108,7 @@ private[spark] class Executor( // Initialize Spark environment (using system properties read above) private val env = { if (!isLocal) { - val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, + val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, conf, isDriver = false, isLocal = false) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) @@ -303,7 +303,7 @@ private[spark] class Executor( * new classes defined by the REPL as the user types code */ private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { - val classUri = System.getProperty("spark.repl.class.uri") + val classUri = conf.getOrElse("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 570a979b56..8ef5019b6c 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,6 +22,7 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.apache.spark.SparkConf /** @@ -37,15 +38,16 @@ trait CompressionCodec { private[spark] object CompressionCodec { - + import org.apache.spark.SparkContext.globalConf def createCodec(): CompressionCodec = { createCodec(System.getProperty( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } def createCodec(codecName: String): CompressionCodec = { - Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] + val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + .getConstructor(classOf[SparkConf]) + ctor.newInstance(globalConf).asInstanceOf[CompressionCodec] } } @@ -53,7 +55,7 @@ private[spark] object CompressionCodec { /** * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. */ -class LZFCompressionCodec extends CompressionCodec { +class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { new LZFOutputStream(s).setFinishBlockOnFlush(true) @@ -67,10 +69,10 @@ class LZFCompressionCodec extends CompressionCodec { * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. */ -class SnappyCompressionCodec extends CompressionCodec { +class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index bec0c83be8..ac29816f19 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source @@ -62,10 +62,11 @@ import org.apache.spark.metrics.source.Source * * [options] is the specific property of this source or sink. */ -private[spark] class MetricsSystem private (val instance: String) extends Logging { +private[spark] class MetricsSystem private (val instance: String, + conf: SparkConf) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf") + val confFile = conf.getOrElse("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] @@ -159,5 +160,6 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem = + new MetricsSystem(instance, conf) } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 703bc6a9ca..3e902f8ac5 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -37,7 +37,7 @@ import scala.concurrent.duration._ import org.apache.spark.util.Utils -private[spark] class ConnectionManager(port: Int) extends Logging { +private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging { class MessageStatus( val message: Message, @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.handler.threads.min","20").toInt, - System.getProperty("spark.core.connection.handler.threads.max","60").toInt, - System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.handler.threads.min", "20").toInt, + conf.getOrElse("spark.core.connection.handler.threads.max", "60").toInt, + conf.getOrElse("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.io.threads.min","4").toInt, - System.getProperty("spark.core.connection.io.threads.max","32").toInt, - System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.io.threads.min", "4").toInt, + conf.getOrElse("spark.core.connection.io.threads.max", "32").toInt, + conf.getOrElse("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.connect.threads.min","1").toInt, - System.getProperty("spark.core.connection.connect.threads.max","8").toInt, - System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.connect.threads.min", "1").toInt, + conf.getOrElse("spark.core.connection.connect.threads.max", "8").toInt, + conf.getOrElse("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() @@ -593,8 +593,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private[spark] object ConnectionManager { + import SparkContext.globalConf + def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 781715108b..4ca3cd390b 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -21,9 +21,9 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object ReceiverTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 777574980f..11c21fc1d5 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object SenderTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { if (args.length < 2) { @@ -33,7 +33,7 @@ private[spark] object SenderTest { val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val manager = new ConnectionManager(0) + val manager = new ConnectionManager(0, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index b1e1576dad..81b3104afd 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -23,20 +23,20 @@ import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.util.CharsetUtil -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.network.ConnectionManagerId import scala.collection.JavaConverters._ import org.apache.spark.storage.BlockId -private[spark] class ShuffleCopier extends Logging { +private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { def getBlock(host: String, port: Int, blockId: BlockId, resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { @@ -107,7 +107,7 @@ private[spark] object ShuffleCopier extends Logging { val tasks = (for (i <- Range(0, threads)) yield { Executors.callable(new Runnable() { def run() { - val copier = new ShuffleCopier() + val copier = new ShuffleCopier(SparkContext.globalConf) copier.getBlock(host, port, blockId, echoResultCollectCallBack) } }) 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 a712ef1c27..9fbe002748 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -75,6 +75,8 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) private[spark] object CheckpointRDD extends Logging { + import SparkContext.{globalConf => conf} + def splitIdToFile(splitId: Int): String = { "part-%05d".format(splitId) } @@ -92,7 +94,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -122,7 +124,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ea45566ad1..f8b1a6932e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -81,6 +81,7 @@ abstract class RDD[T: ClassTag]( def this(@transient oneParent: RDD[_]) = this(oneParent.context , List(new OneToOneDependency(oneParent))) + private[spark] def conf = sc.conf // ======================================================================= // Methods that should be implemented by subclasses of RDD // ======================================================================= diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 60927831a1..3f55cd5642 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - def this() = this(System.getProperty("user.name", ""), + def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) private val logDir = diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 356fe56bf3..9002d33cda 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} import java.util.{NoSuchElementException, Properties} -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import scala.xml.XML @@ -49,10 +49,10 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) } } -private[spark] class FairSchedulableBuilder(val rootPool: Pool) +private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file")) + val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file")) val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" 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 66ab8ea4cd..7e231ec44c 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 @@ -49,11 +49,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler with Logging { + val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -90,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -108,7 +109,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() @@ -119,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (System.getProperty("spark.speculation", "false").toBoolean) { + if (conf.getOrElse("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, 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 bf494aa64d..398b0cefbf 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 @@ -50,15 +50,16 @@ private[spark] class ClusterTaskSetManager( extends TaskSetManager with Logging { + val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt + val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -117,7 +118,7 @@ private[spark] class ClusterTaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception @@ -677,14 +678,14 @@ private[spark] class ClusterTaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = System.getProperty("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - System.getProperty("spark.locality.wait.process", defaultWait).toLong + conf.getOrElse("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - System.getProperty("spark.locality.wait.node", defaultWait).toLong + conf.getOrElse("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - System.getProperty("spark.locality.wait.rack", defaultWait).toLong + conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7e22c843bf..40555903ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.util.Try import akka.actor._ import akka.pattern.ask @@ -46,8 +47,8 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - - private val timeout = AkkaUtils.askTimeout + val conf = scheduler.sc.conf + private val timeout = AkkaUtils.askTimeout(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -61,7 +62,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac 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 + val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -162,7 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac override def start() { val properties = new ArrayBuffer[(String, String)] - val iterator = System.getProperties.entrySet.iterator + val iterator = scheduler.sc.conf.getAllConfiguration while (iterator.hasNext) { val entry = iterator.next val (key, value) = (entry.getKey.toString, entry.getValue.toString) @@ -170,6 +171,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac properties += ((key, value)) } } + //TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } @@ -208,7 +210,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac driverActor ! KillTask(taskId, executorId) } - override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism")) + override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) // Called by subclasses when notified of a lost worker diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index e8fecec4a6..d01329b2b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -31,13 +31,13 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt override def start() { super.start() val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() 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 7127a72d6d..d6b8ac2d57 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 @@ -36,14 +36,14 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt override def start() { super.start() // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command( @@ -52,7 +52,7 @@ private[spark] class SparkDeploySchedulerBackend( val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, "http://" + sc.ui.appUIAddress) - client = new Client(sc.env.actorSystem, masters, appDesc, this) + client = new Client(sc.env.actorSystem, masters, appDesc, this, conf) client.start() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index e68c527713..ff6cc37f1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -31,7 +31,8 @@ import org.apache.spark.util.Utils */ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt + + private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") 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 84fe3094cc..2a3b0e15f7 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 @@ -61,7 +61,7 @@ private[spark] class CoarseMesosSchedulerBackend( var driver: SchedulerDriver = null // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt // Cores we have acquired with each Mesos task ID val coresByTaskId = new HashMap[Int, Int] @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 @@ -121,10 +121,10 @@ private[spark] class CoarseMesosSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val uri = System.getProperty("spark.executor.uri") + val uri = conf.get("spark.executor.uri") if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 50cbc2ca92..9bb92b4f01 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -101,7 +101,7 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = System.getProperty("spark.executor.uri") + val uri = sc.conf.get("spark.executor.uri") if (uri == null) { command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) } else { @@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index 01e95162c0..6069c1db3a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -80,6 +80,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val with Logging { val env = SparkEnv.get + val conf = env.conf val attemptId = new AtomicInteger var dagScheduler: DAGScheduler = null @@ -91,7 +92,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -106,7 +107,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index e748c2275d..17cec81038 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -25,18 +25,20 @@ import com.esotericsoftware.kryo.{KryoException, Kryo} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} -import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.{SparkContext, SparkConf, SerializableWritable, Logging} import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ +import scala.util.Try /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { + private val conf = SparkContext.globalConf private val bufferSize = { - System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +50,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -58,7 +60,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow the user to register their own classes by setting spark.kryo.registrator try { - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls => logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index e51c5b30a3..ee2ae471a9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -312,7 +312,7 @@ object BlockFetcherIterator { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.host)) val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier + val cpier = new ShuffleCopier(blockManager.conf) cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 19a025a329..ffd166e93a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -30,7 +30,7 @@ import scala.concurrent.duration._ import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -43,12 +43,13 @@ private[spark] class BlockManager( actorSystem: ActorSystem, val master: BlockManagerMaster, val defaultSerializer: Serializer, - maxMemory: Long) + maxMemory: Long, + val conf: SparkConf) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir"))) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -57,12 +58,12 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0) + val connectionManager = new ConnectionManager(0, conf) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( @@ -71,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - System.getProperty("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties @@ -115,8 +116,8 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + serializer: Serializer, conf: SparkConf) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf) } /** @@ -439,7 +440,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -465,7 +466,8 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream) + val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } /** @@ -856,19 +858,19 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - + import org.apache.spark.SparkContext.{globalConf => conf} val ID_GENERATOR = new IdGenerator def getMaxMemoryFromSystemProperties: Long = { - val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = - System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that 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 e1d68ef592..fde7d63a68 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,19 +23,20 @@ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { +class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], + conf: SparkConf) 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 + val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 21022e1cfb..05502e4451 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -36,7 +36,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { +class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = @@ -48,14 +48,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val akkaTimeout = AkkaUtils.askTimeout + private val akkaTimeout = AkkaUtils.askTimeout(conf) initLogging() - val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs", + val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong - val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", + val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong var timeoutCheckingTask: Cancellable = null diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index b4451fc7b8..61e63c60d5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -74,7 +74,8 @@ class DiskBlockObjectWriter( file: File, serializer: Serializer, bufferSize: Int, - compressStream: OutputStream => OutputStream) + compressStream: OutputStream => OutputStream, + syncWrites: Boolean) extends BlockObjectWriter(blockId) with Logging { @@ -97,8 +98,6 @@ class DiskBlockObjectWriter( override def flush() = out.flush() } - private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean - /** The file channel, used for repositioning / truncating the file. */ private var channel: FileChannel = null private var bs: OutputStream = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index fcd2e97982..8f528babd4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index e828e1d1c5..850d3178dd 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -27,6 +27,8 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup +import scala.util.Try +import org.apache.spark.SparkConf /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -59,12 +61,13 @@ private[spark] trait ShuffleWriterGroup { */ private[spark] class ShuffleBlockManager(blockManager: BlockManager) { + def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused 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 a8db37ded1..b3b3893393 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,6 +22,7 @@ import akka.actor._ import java.util.concurrent.ArrayBlockingQueue import util.Random import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.SparkContext /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -91,11 +92,12 @@ private[spark] object ThreadingTest { def main(args: Array[String]) { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") + val conf = SparkContext.globalConf val serializer = new KryoSerializer val blockManagerMaster = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f1d86c0221..0ce8d9c8c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt + val port = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None var server: Option[Server] = None diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index fcd1b518d0..14751e8e8e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -30,6 +30,8 @@ import org.apache.spark.scheduler.SchedulingMode * Usage: ./run spark.ui.UIWorkloadGenerator [master] */ private[spark] object UIWorkloadGenerator { + + import SparkContext.{globalConf => conf} val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 @@ -43,7 +45,7 @@ private[spark] object UIWorkloadGenerator { val appName = "Spark UI Tester" if (schedulingMode == SchedulingMode.FAIR) { - System.setProperty("spark.scheduler.mode", "FAIR") + conf.set("spark.scheduler.mode", "FAIR") } val sc = new SparkContext(master, appName) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index c5bf2acc9e..b637d37517 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -63,7 +63,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(System.getProperty("path.separator", ":")) + .split(sc.conf.getOrElse("path.separator", ":")) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 6b854740d6..f01a1380b9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() 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 1c8b51b8bc..76febd5702 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -21,6 +21,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory +import org.apache.spark.SparkConf /** * Various utility classes for working with Akka. @@ -37,22 +38,22 @@ private[spark] object AkkaUtils { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false) - : (ActorSystem, Int) = { + def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt - val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt - val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt + conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -87,7 +88,7 @@ private[spark] object AkkaUtils { } /** Returns the default Spark timeout to use for Akka ask operations. */ - def askTimeout: FiniteDuration = { - Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds") + def askTimeout(conf: SparkConf): FiniteDuration = { + Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 7b41ef89f1..bf71d17a21 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.util.{TimerTask, Timer} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} /** @@ -66,21 +66,21 @@ object MetadataCleanerType extends Enumeration { } object MetadataCleaner { - + private val conf = SparkContext.globalConf // using only sys props for now : so that workers can also get to it while preserving earlier behavior. - def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt + def getDelaySeconds = conf.getOrElse("spark.cleaner.ttl", "3500").toInt //TODO: this is to fix tests for time being def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt + conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt } def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { - System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString) + conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } def setDelaySeconds(delay: Int, resetAll: Boolean = true) { // override for all ? - System.setProperty("spark.cleaner.ttl", delay.toString) + conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index a25b37a2a9..1407c39bfb 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -30,7 +30,7 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, SparkContext, Logging} /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -41,6 +41,7 @@ import org.apache.spark.Logging */ private[spark] object SizeEstimator extends Logging { + private def conf = SparkContext.globalConf // Sizes of primitive types private val BYTE_SIZE = 1 private val BOOLEAN_SIZE = 1 @@ -90,8 +91,8 @@ private[spark] object SizeEstimator extends Logging { } private def getIsCompressedOops : Boolean = { - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean + if (conf.getOrElse("spark.test.useCompressedOops", null) != null) { + return conf.get("spark.test.useCompressedOops").toBoolean } try { 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 3f7858d2de..fd5888e525 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.{SparkContext, SparkException, Logging} /** @@ -44,6 +44,7 @@ import org.apache.spark.{SparkException, Logging} */ private[spark] object Utils extends Logging { + private lazy val conf = SparkContext.globalConf /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -312,7 +313,7 @@ private[spark] object Utils extends Logging { * multiple paths. */ def getLocalDir: String = { - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } /** @@ -398,7 +399,7 @@ private[spark] object Utils extends Logging { } def localHostPort(): String = { - val retval = System.getProperty("spark.hostPort", null) + val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 271dc905bc..10b8b441fd 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { - + private val conf = new SparkConf test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) assert(MapOutputTracker.compressSize(1L) === 1) @@ -48,14 +48,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -74,7 +74,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -96,16 +96,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) - val masterTracker = new MapOutputTrackerMaster() + val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = Left(actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) - val slaveTracker = new MapOutputTracker() + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) + val slaveTracker = new MapOutputTracker(conf) slaveTracker.trackerActor = Right(slaveSystem.actorSelection( "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 7181333adf..4ecdde0001 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -19,17 +19,19 @@ package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.SparkConf class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + var conf: SparkConf = null before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf", filePath) + conf = new SparkConf } test("MetricsSystem with default config") { - val metricsSystem = MetricsSystem.createMetricsSystem("default") + val metricsSystem = MetricsSystem.createMetricsSystem("default", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks @@ -39,7 +41,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { } test("MetricsSystem with sources add") { - val metricsSystem = MetricsSystem.createMetricsSystem("test") + val metricsSystem = MetricsSystem.createMetricsSystem("test", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 706d84a58b..2aa259daf3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,21 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Map, HashMap} - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import org.apache.spark.LocalSparkContext -import org.apache.spark.MapOutputTrackerMaster -import org.apache.spark.SparkContext -import org.apache.spark.Partition -import org.apache.spark.TaskContext -import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency} -import org.apache.spark.{FetchFailed, Success, TaskEndReason} +import scala.Tuple2 +import scala.collection.mutable.{HashMap, Map} + +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} +import org.scalatest.{BeforeAndAfter, FunSuite} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler @@ -46,7 +39,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} * and capturing the resulting TaskSets from the mock TaskScheduler. */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { @@ -74,7 +67,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null) { + val blockManagerMaster = new BlockManagerMaster(null, conf) { override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). @@ -99,7 +92,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSets.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster() + mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 002368ff55..dd122615ad 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -95,7 +95,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) + val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) joblogger.getJobIDtoPrintWriter.size should be (1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 95d3553d91..34d2e4cb8c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -169,7 +169,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.scheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() assert(rootPool.getSchedulableByName("default") != null) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index bb28a31a99..2bb827c022 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -81,8 +81,8 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} - - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + private val conf = new SparkConf + val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong test("TaskSet with no preferences") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 27c2d53361..618fae7c16 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.{SparkConf, LocalSparkContext, SparkContext, SparkEnv} import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} import org.apache.spark.storage.TaskResultBlockId 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 5b4d63b954..4ef5538951 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,8 +31,10 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.{SparkConf, SparkContext} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -42,7 +44,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldHeartBeat: String = null // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - System.setProperty("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.mb", "1") val serializer = new KryoSerializer // Implicitly convert strings to BlockIds for test clarity. @@ -50,22 +52,23 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf) this.actorSystem = actorSystem - System.setProperty("spark.driver.port", boundPort.toString) - System.setProperty("spark.hostPort", "localhost:" + boundPort) + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") + System.setProperty("os.arch", "amd64") + conf.set("os.arch", "amd64") + conf.set("spark.test.useCompressedOops", "true") + conf.set("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() // Set some value ... - System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111) + conf.set("spark.hostPort", Utils.localHostName() + ":" + 1111) } after { @@ -86,13 +89,13 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master = null if (oldArch != null) { - System.setProperty("os.arch", oldArch) + conf.set("os.arch", oldArch) } else { System.clearProperty("os.arch") } if (oldOops != null) { - System.setProperty("spark.test.useCompressedOops", oldOops) + conf.set("spark.test.useCompressedOops", oldOops) } else { System.clearProperty("spark.test.useCompressedOops") } @@ -133,7 +136,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -163,8 +166,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -179,7 +182,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -227,7 +230,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -261,7 +264,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -277,7 +280,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -296,7 +299,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -333,7 +336,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -352,7 +355,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -371,7 +374,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -390,7 +393,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -413,7 +416,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -426,7 +429,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -441,7 +444,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -456,7 +459,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -471,7 +474,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -486,7 +489,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -511,7 +514,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -535,7 +538,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -581,7 +584,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500) + store = new BlockManager("", actorSystem, master, serializer, 500, conf) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -591,53 +594,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { - System.setProperty("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "true") + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "false") + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "true") + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "false") + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "true") + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "false") + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -651,7 +654,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200) + store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200, conf) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 070982e798..f940448abd 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.storage -import java.io.{FileWriter, File} +import java.io.{File, FileWriter} import scala.collection.mutable import com.google.common.io.Files +import org.apache.spark.SparkConf import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import scala.util.Try +import akka.actor.{Props, ActorSelection, ActorSystem} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { - + private val testConf = new SparkConf val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() val rootDir1 = Files.createTempDir() @@ -36,10 +39,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. val consolidateProp = "spark.shuffle.consolidateFiles" - val oldConsolidate = Option(System.getProperty(consolidateProp)) - System.setProperty(consolidateProp, "true") + val oldConsolidate = Try(testConf.get(consolidateProp)).toOption + testConf.set(consolidateProp, "true") val shuffleBlockManager = new ShuffleBlockManager(null) { + override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) } diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 5aff26f9fc..a5facd5bbd 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkContext class DummyClass1 {} @@ -139,7 +140,8 @@ class SizeEstimatorSuite test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") - + SparkContext.globalConf.set("os.arch", "amd64") + SparkContext.globalConf.set("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 72b5c7b88e..12c430be27 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -36,16 +36,18 @@ object WikipediaPageRank { System.err.println("Usage: WikipediaPageRank ") System.exit(-1) } - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) val threshold = args(1).toDouble val numPartitions = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRank") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph val input = sc.textFile(inputFile) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index ddf6855325..5bf0b7a24a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -34,15 +34,19 @@ object WikipediaPageRankStandalone { System.err.println("Usage: WikipediaPageRankStandalone ") System.exit(-1) } + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") - System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) val threshold = args(1).toDouble val numIterations = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRankStandalone") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + + val sc = new SparkContext(sparkConf) val input = sc.textFile(inputFile) val partitioner = new HashPartitioner(sc.defaultParallelism) 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 50e3f9639c..2402409e6e 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 @@ -26,6 +26,7 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.actorRef2Scala +import org.apache.spark.SparkConf import org.apache.spark.streaming.Seconds import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions @@ -116,7 +117,7 @@ object FeederActor { val Seq(host, port) = args.toSeq - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1 val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") println("Feeder started as:" + feeder) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 36853acab5..2f2d106f86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -578,14 +578,13 @@ object ALS { val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false val alpha = if (args.length >= 8) args(7).toDouble else 1 val blocks = if (args.length == 9) args(8).toInt else -1 - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "8") - System.setProperty("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS") + sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + sc.conf.set("spark.kryo.referenceTracking", "false") + sc.conf.set("spark.kryoserializer.buffer.mb", "8") + sc.conf.set("spark.locality.wait", "10000") + val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index eeeca3ea8a..433268a1dd 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -61,13 +61,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var amClient: AMRMClient[ContainerRequest] = _ // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -138,10 +138,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while (!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -265,7 +265,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -343,7 +343,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 94678815e8..a322f60864 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -499,7 +499,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 70be15d0a3..41ac292249 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,7 +35,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index bc31bb2eb0..f7d73f0d83 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -136,8 +136,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c27257cda4..71d1cbd416 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -254,8 +254,8 @@ private[yarn] class YarnAllocationHandler( } else { val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c78..6feaaff014 100644 --- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String]( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab96cfa18b..ffb54a24ac 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -244,7 +244,8 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1", - "com.twitter" % "chill-java" % "0.3.1" + "com.twitter" % "chill-java" % "0.3.1", + "com.typesafe" % "config" % "1.0.2" ) ) 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 523fd1222d..b2f499e637 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -930,9 +930,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def createSparkContext(): SparkContext = { val uri = System.getenv("SPARK_EXECUTOR_URI") - if (uri != null) { - System.setProperty("spark.executor.uri", uri) - } val master = this.master match { case Some(m) => m case None => { @@ -942,6 +939,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) + if (uri != null) { + sparkContext.conf.set("spark.executor.uri", uri) + } + sparkContext.conf.set("spark.repl.class.uri", intp.classServer.uri) echo("Created spark context..") sparkContext } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index e1455ef8a1..0d412e4478 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -34,10 +34,8 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.HttpServer +import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging} import org.apache.spark.util.Utils -import org.apache.spark.SparkEnv -import org.apache.spark.Logging // /** directory to save .class files to */ // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { @@ -91,7 +89,7 @@ import org.apache.spark.Logging /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) + val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { @@ -112,7 +110,6 @@ import org.apache.spark.Logging // Start the classServer and store its URI in a spark system property // (which will be passed to executors so that they can connect to it) classServer.start() - System.setProperty("spark.repl.class.uri", classServer.uri) if (SPARK_DEBUG_REPL) { echo("Class server started, URI = " + classServer.uri) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 9271914eb5..b8e1427a21 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -34,7 +34,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master val framework = ssc.sc.appName - val sparkHome = ssc.sc.sparkHome + val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars val environment = ssc.sc.environment val graph = ssc.graph @@ -42,6 +42,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() val delaySeconds = MetadataCleaner.getDelaySeconds + val sparkConf = ssc.sc.conf def validate() { assert(master != null, "Checkpoint.master is null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index ed892e33e6..1d23713c80 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -26,7 +26,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt + val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { new CheckpointWriter(ssc.checkpointDir) @@ -34,7 +34,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { null } - val clockClass = System.getProperty( + val clockClass = ssc.sc.conf.getOrElse( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, @@ -73,7 +73,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } 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 d2c4fdee65..76744223e1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -115,7 +115,7 @@ class StreamingContext private ( protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment) + new SparkContext(cp_.sparkConf, cp_.environment) } else { sc_ } 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 d5ae8aef92..8bf761b8cb 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 @@ -175,8 +175,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = System.getProperty("spark.driver.host", "localhost") - val port = System.getProperty("spark.driver.port", "7077").toInt + val ip = env.conf.getOrElse("spark.driver.host", "localhost") + val port = env.conf.getOrElse("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -213,7 +213,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) 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 e81287b44e..315bd5443c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.streaming.util.ManualClock */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") before { FileUtils.deleteDirectory(new File(checkpointDir)) @@ -69,7 +69,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) @@ -135,13 +135,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Restart stream computation from the new checkpoint file to see whether that file has // correct checkpoint data + conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc = new StreamingContext(checkpointDir) stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]") assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure") // Adjust manual clock time as if it is being restarted after a delay - System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc.start() advanceTimeWithRealDelay(ssc, 4) ssc.stop() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7dc82decef..da8f135dd7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -53,7 +53,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" before { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { @@ -68,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] val outputStream = new TestOutputStream(networkStream, outputBuffer) @@ -113,7 +113,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("flume input stream") { // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] @@ -162,11 +162,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("file input stream") { // Disable manual clock as FileInputDStream does not work with manual clock - System.clearProperty("spark.streaming.clock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") // Set up the streaming context and input streams val testDir = Files.createTempDir() - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] def output = outputBuffer.flatMap(x => x) @@ -207,7 +207,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(testDir) // Enable manual clock back again for other tests - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } @@ -218,7 +218,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] @@ -262,7 +262,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } test("kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val topics = Map("my-topic" -> 1) val test1 = ssc.kafkaStream("localhost:12345", "group", topics) val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) @@ -285,7 +285,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { MultiThreadTestReceiver.haveAllThreadsFinished = false // set up the network stream using the test receiver - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.networkStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 2f34e812a1..d1cab0c609 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -28,7 +28,7 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.rdd.RDD /** @@ -130,6 +130,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Whether to actually wait in real time before changing manual clock def actuallyWait = false + def conf = new SparkConf().setMasterUrl(master).setAppName(framework).set("spark.cleaner.ttl", "3600") /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. @@ -139,9 +140,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { operation: DStream[U] => DStream[V], numPartitions: Int = numInputPartitions ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } @@ -165,9 +166,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W] ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 240ed8b32a..1dd38dd13e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -58,13 +58,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true // default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -165,10 +165,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while(!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -294,7 +294,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -377,7 +377,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 79dd038065..29892e98e3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -461,7 +461,7 @@ object Client { def main(argStrings: Array[String]) { // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") + conf.set("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) @@ -483,7 +483,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index b3a7886d93..617289f568 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,7 +33,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 69038844bb..c1e79cbe66 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -162,8 +162,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) 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 9ab2073529..4c9fee5695 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 @@ -239,7 +239,7 @@ private[yarn] class YarnAllocationHandler( // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c78..6feaaff014 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String]( -- cgit v1.2.3 From 6e43039614ed1ec55a134fb82fb3e8d4e80996ef Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 26 Dec 2013 18:02:49 -0800 Subject: Refactored streaming project to separate out the twitter functionality. --- .../streaming/examples/TwitterAlgebirdCMS.scala | 2 + .../streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../streaming/examples/TwitterPopularTags.scala | 1 + .../twitter/StreamingContextWithTwitter.scala | 27 ++++++ .../streaming/twitter/TwitterInputDStream.scala | 101 +++++++++++++++++++++ .../apache/spark/streaming/twitter/package.scala | 7 ++ project/SparkBuild.scala | 13 ++- .../apache/spark/streaming/StreamingContext.scala | 6 +- .../streaming/api/java/JavaStreamingContext.scala | 8 +- .../streaming/dstream/TwitterInputDStream.scala | 99 -------------------- 10 files changed, 158 insertions(+), 108 deletions(-) create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala (limited to 'examples') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 35b6329ab3..a00b3bde6e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -23,6 +23,8 @@ import com.twitter.algebird._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.SparkContext._ +import org.apache.spark.streaming.twitter._ + /** * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute * windowed and global Top-K estimates of user IDs occurring in a Twitter stream. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 8bfde2a829..82156060a8 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.storage.StorageLevel import com.twitter.algebird.HyperLogLog._ import com.twitter.algebird.HyperLogLogMonoid -import org.apache.spark.streaming.dstream.TwitterInputDStream +import org.apache.spark.streaming.twitter._ /** * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 27aa6b14bf..c4ded5e071 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples import org.apache.spark.streaming.{Seconds, StreamingContext} import StreamingContext._ import org.apache.spark.SparkContext._ +import org.apache.spark.streaming.twitter._ /** * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala new file mode 100644 index 0000000000..fe66e28ce6 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala @@ -0,0 +1,27 @@ +package org.apache.spark.streaming.twitter + +import twitter4j.Status +import twitter4j.auth.Authorization +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + + +class StreamingContextWithTwitter(ssc: StreamingContext) { + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Option[Authorization] = None, + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = { + val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala new file mode 100644 index 0000000000..97e48ebeca --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -0,0 +1,101 @@ +/* + * 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.twitter + +import java.util.prefs.Preferences +import twitter4j._ +import twitter4j.auth.Authorization +import twitter4j.conf.ConfigurationBuilder +import twitter4j.conf.PropertyConfiguration +import twitter4j.auth.OAuthAuthorization +import twitter4j.auth.AccessToken +import org.apache.spark._ +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream._ +import org.apache.spark.storage.StorageLevel + +/* A stream of Twitter statuses, potentially filtered by one or more keywords. +* +* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. +* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is +* such that this may return a sampled subset of all tweets during each interval. +* +* If no Authorization object is provided, initializes OAuth authorization using the system +* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. +*/ +private[streaming] +class TwitterInputDStream( + @transient ssc_ : StreamingContext, + twitterAuth: Option[Authorization], + filters: Seq[String], + storageLevel: StorageLevel + ) extends NetworkInputDStream[Status](ssc_) { + + private def createOAuthAuthorization(): Authorization = { + new OAuthAuthorization(new ConfigurationBuilder().build()) + } + + private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) + + override def getReceiver(): NetworkReceiver[Status] = { + new TwitterReceiver(authorization, filters, storageLevel) + } +} + +private[streaming] +class TwitterReceiver( + twitterAuth: Authorization, + filters: Seq[String], + storageLevel: StorageLevel + ) extends NetworkReceiver[Status] { + + var twitterStream: TwitterStream = _ + lazy val blockGenerator = new BlockGenerator(storageLevel) + + protected override def onStart() { + blockGenerator.start() + twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) + twitterStream.addListener(new StatusListener { + def onStatus(status: Status) = { + blockGenerator += status + } + // Unimplemented + def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} + def onTrackLimitationNotice(i: Int) {} + def onScrubGeo(l: Long, l1: Long) {} + def onStallWarning(stallWarning: StallWarning) {} + def onException(e: Exception) { stopOnError(e) } + }) + + val query: FilterQuery = new FilterQuery + if (filters.size > 0) { + query.track(filters.toArray) + twitterStream.filter(query) + } else { + twitterStream.sample() + } + logInfo("Twitter receiver started") + } + + protected override def onStop() { + blockGenerator.stop() + twitterStream.shutdown() + logInfo("Twitter receiver stopped") + } +} + diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala new file mode 100644 index 0000000000..89c202a730 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala @@ -0,0 +1,7 @@ +package org.apache.spark.streaming + +package object twitter { + implicit def enrichMyStreamingContext(ssc: StreamingContext): StreamingContextWithTwitter = { + new StreamingContextWithTwitter(ssc) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7bcbd90bd3..f9ff781f38 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -50,7 +50,7 @@ object SparkBuild extends Build { .dependsOn(core, bagel, mllib) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, bagel, streaming) + .dependsOn(core, mllib, bagel, streaming, externalTwitter) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) @@ -60,6 +60,8 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) + lazy val externalTwitter = Project("streaming-twitter", file("external/twitter"), settings = twitterSettings) dependsOn(streaming) + lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) @@ -313,7 +315,7 @@ object SparkBuild extends Build { excludeAll(excludeNetty), "org.eclipse.paho" % "mqtt-client" % "0.4.0", "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), - "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), + // "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) ) ) @@ -354,4 +356,11 @@ object SparkBuild extends Build { case _ => MergeStrategy.first } ) + + def twitterSettings() = streamingSettings ++ Seq( + name := "spark-twitter", + libraryDependencies ++= Seq( + "org.twitter4j" % "twitter4j-stream" % "3.0.3" 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 41da028a3c..25b9b70b2c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -45,8 +45,8 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path -import twitter4j.Status -import twitter4j.auth.Authorization +//import twitter4j.Status +//import twitter4j.auth.Authorization import org.apache.spark.streaming.scheduler._ import akka.util.ByteString @@ -414,6 +414,7 @@ class StreamingContext private ( fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) } + /* /** * Create a input stream that returns tweets received from Twitter. * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth @@ -431,6 +432,7 @@ class StreamingContext private ( registerInputStream(inputStream) inputStream } + */ /** * Create an input stream from a queue of RDDs. In each batch, 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 78d318cf27..b32cfbb677 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 @@ -25,13 +25,13 @@ import scala.collection.JavaConversions._ import scala.reflect.ClassTag import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import twitter4j.Status +//import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe import akka.util.ByteString -import twitter4j.auth.Authorization +//import twitter4j.auth.Authorization import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -338,7 +338,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { ssc.flumeStream(hostname, port) } - + /* /** * Create a input stream that returns tweets received from Twitter. * @param twitterAuth Twitter4J Authorization object @@ -409,7 +409,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def twitterStream(): JavaDStream[Status] = { ssc.twitterStream() } - + */ /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala deleted file mode 100644 index 387e15b0e6..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala +++ /dev/null @@ -1,99 +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._ -import org.apache.spark.streaming._ -import storage.StorageLevel -import twitter4j._ -import twitter4j.auth.Authorization -import java.util.prefs.Preferences -import twitter4j.conf.ConfigurationBuilder -import twitter4j.conf.PropertyConfiguration -import twitter4j.auth.OAuthAuthorization -import twitter4j.auth.AccessToken - -/* A stream of Twitter statuses, potentially filtered by one or more keywords. -* -* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. -* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is -* such that this may return a sampled subset of all tweets during each interval. -* -* If no Authorization object is provided, initializes OAuth authorization using the system -* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. -*/ -private[streaming] -class TwitterInputDStream( - @transient ssc_ : StreamingContext, - twitterAuth: Option[Authorization], - filters: Seq[String], - storageLevel: StorageLevel - ) extends NetworkInputDStream[Status](ssc_) { - - private def createOAuthAuthorization(): Authorization = { - new OAuthAuthorization(new ConfigurationBuilder().build()) - } - - private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - - override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(authorization, filters, storageLevel) - } -} - -private[streaming] -class TwitterReceiver( - twitterAuth: Authorization, - filters: Seq[String], - storageLevel: StorageLevel - ) extends NetworkReceiver[Status] { - - var twitterStream: TwitterStream = _ - lazy val blockGenerator = new BlockGenerator(storageLevel) - - protected override def onStart() { - blockGenerator.start() - twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) - twitterStream.addListener(new StatusListener { - def onStatus(status: Status) = { - blockGenerator += status - } - // Unimplemented - def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} - def onTrackLimitationNotice(i: Int) {} - def onScrubGeo(l: Long, l1: Long) {} - def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { stopOnError(e) } - }) - - val query: FilterQuery = new FilterQuery - if (filters.size > 0) { - query.track(filters.toArray) - twitterStream.filter(query) - } else { - twitterStream.sample() - } - logInfo("Twitter receiver started") - } - - protected override def onStop() { - blockGenerator.stop() - twitterStream.shutdown() - logInfo("Twitter receiver stopped") - } -} -- cgit v1.2.3 From 642029e7f43322f84abe4f7f36bb0b1b95d8101d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 28 Dec 2013 17:13:15 -0500 Subject: Various fixes to configuration code - Got rid of global SparkContext.globalConf - Pass SparkConf to serializers and compression codecs - Made SparkConf public instead of private[spark] - Improved API of SparkContext and SparkConf - Switched executor environment vars to be passed through SparkConf - Fixed some places that were still using system properties - Fixed some tests, though others are still failing This still fails several tests in core, repl and streaming, likely due to properties not being set or cleared correctly (some of the tests run fine in isolation). --- .../main/scala/org/apache/spark/Accumulators.scala | 8 +- .../scala/org/apache/spark/MapOutputTracker.scala | 4 +- .../main/scala/org/apache/spark/Partitioner.scala | 6 +- .../main/scala/org/apache/spark/SparkConf.scala | 158 ++++++++++++++++----- .../main/scala/org/apache/spark/SparkContext.scala | 138 ++++++++++-------- .../src/main/scala/org/apache/spark/SparkEnv.scala | 11 +- .../apache/spark/api/java/JavaSparkContext.scala | 15 +- .../org/apache/spark/api/python/PythonRDD.scala | 6 +- .../org/apache/spark/broadcast/HttpBroadcast.scala | 33 +++-- .../apache/spark/broadcast/TorrentBroadcast.scala | 38 ++--- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache/spark/deploy/LocalSparkCluster.scala | 7 +- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 14 +- .../apache/spark/deploy/client/TestClient.scala | 9 +- .../org/apache/spark/deploy/master/Master.scala | 36 ++--- .../deploy/master/SparkZooKeeperSession.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 2 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 2 +- .../org/apache/spark/deploy/worker/Worker.scala | 14 +- .../executor/CoarseGrainedExecutorBackend.scala | 4 +- .../scala/org/apache/spark/executor/Executor.scala | 17 +-- .../org/apache/spark/io/CompressionCodec.scala | 13 +- .../apache/spark/network/ConnectionManager.scala | 4 +- .../org/apache/spark/network/ReceiverTest.scala | 12 +- .../org/apache/spark/network/SenderTest.scala | 16 +-- .../apache/spark/network/netty/ShuffleCopier.scala | 6 +- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 7 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/SubtractedRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 5 +- .../apache/spark/scheduler/InputFormatInfo.scala | 14 +- .../org/apache/spark/scheduler/JobLogger.scala | 2 +- .../org/apache/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/SchedulableBuilder.scala | 2 +- .../apache/spark/scheduler/ShuffleMapTask.scala | 6 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 8 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 12 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 9 +- .../spark/scheduler/cluster/SchedulerBackend.scala | 3 - .../scheduler/cluster/SimrSchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../spark/scheduler/cluster/TaskResultGetter.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 6 +- .../cluster/mesos/MesosSchedulerBackend.scala | 6 +- .../spark/scheduler/local/LocalScheduler.scala | 2 +- .../apache/spark/serializer/JavaSerializer.scala | 3 +- .../apache/spark/serializer/KryoSerializer.scala | 13 +- .../spark/serializer/SerializerManager.scala | 12 +- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 46 +++--- .../apache/spark/storage/BlockManagerMaster.scala | 4 +- .../spark/storage/BlockManagerMasterActor.scala | 4 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockManager.scala | 9 +- .../apache/spark/storage/StoragePerfTester.scala | 2 +- .../org/apache/spark/storage/ThreadingTest.scala | 6 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 17 ++- .../org/apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 4 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 18 +-- .../org/apache/spark/util/MetadataCleaner.scala | 33 +++-- .../org/apache/spark/util/SizeEstimator.scala | 17 +-- .../main/scala/org/apache/spark/util/Utils.scala | 14 +- .../apache/spark/io/CompressionCodecSuite.scala | 8 +- .../cluster/ClusterTaskSetManagerSuite.scala | 2 +- .../spark/serializer/KryoSerializerSuite.scala | 14 +- .../apache/spark/storage/BlockManagerSuite.scala | 8 +- .../org/apache/spark/util/SizeEstimatorSuite.scala | 2 - .../spark/examples/bagel/WikipediaPageRank.scala | 4 +- .../bagel/WikipediaPageRankStandalone.scala | 4 +- .../apache/spark/mllib/recommendation/ALS.scala | 10 +- .../spark/deploy/yarn/ApplicationMaster.scala | 44 +++--- .../org/apache/spark/deploy/yarn/Client.scala | 38 ++--- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../scala/org/apache/spark/repl/SparkILoop.scala | 16 ++- .../scala/org/apache/spark/repl/SparkIMain.scala | 4 +- .../org/apache/spark/streaming/Checkpoint.scala | 22 +-- .../scala/org/apache/spark/streaming/DStream.scala | 2 +- .../org/apache/spark/streaming/Scheduler.scala | 10 +- .../apache/spark/streaming/StreamingContext.scala | 25 ++-- .../streaming/dstream/NetworkInputDStream.scala | 6 +- .../spark/streaming/util/RawTextSender.scala | 4 +- .../apache/spark/streaming/InputStreamsSuite.scala | 6 +- .../org/apache/spark/streaming/TestSuiteBase.scala | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 56 ++++---- .../org/apache/spark/deploy/yarn/Client.scala | 50 +++---- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- 88 files changed, 692 insertions(+), 536 deletions(-) (limited to 'examples') diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6e922a612a..5f73d234aa 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -41,7 +41,7 @@ class Accumulable[R, T] ( @transient initialValue: R, param: AccumulableParam[R, T]) extends Serializable { - + val id = Accumulators.newId @transient private var value_ = initialValue // Current value on master val zero = param.zero(initialValue) // Zero value to be passed to workers @@ -113,7 +113,7 @@ class Accumulable[R, T] ( def setValue(newValue: R) { this.value = newValue } - + // Called by Java when deserializing an object private def readObject(in: ObjectInputStream) { in.defaultReadObject() @@ -177,7 +177,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser def zero(initialValue: R): R = { // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. // Instead we'll serialize it to a buffer and load it back. - val ser = new JavaSerializer().newInstance() + val ser = new JavaSerializer(new SparkConf(false)).newInstance() val copy = ser.deserialize[R](ser.serialize(initialValue)) copy.clear() // In case it contained stuff copy @@ -215,7 +215,7 @@ private object Accumulators { val originals = Map[Long, Accumulable[_, _]]() val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() var lastId: Long = 0 - + def newId: Long = synchronized { lastId += 1 return lastId diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4520edb10d..cdae167aef 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -65,7 +65,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { protected val epochLock = new java.lang.Object private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup) + new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) // Send a message to the trackerActor and get its result within a default timeout, or // throw a SparkException if this fails. @@ -129,7 +129,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { if (fetchedStatuses == null) { // We won the race to fetch the output locs; do so logInfo("Doing the fetch; tracker actor = " + trackerActor) - val hostPort = Utils.localHostPort() + val hostPort = Utils.localHostPort(conf) // This try-finally prevents hangs due to timeouts: try { val fetchedBytes = diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 04c1eedfeb..7cb545a6be 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -32,8 +32,6 @@ abstract class Partitioner extends Serializable { } object Partitioner { - - import SparkContext.{globalConf => conf} /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -54,7 +52,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (conf.getOrElse("spark.default.parallelism", null) != null) { + if (rdd.context.conf.getOrElse("spark.default.parallelism", null) != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) @@ -92,7 +90,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { class RangePartitioner[K <% Ordered[K]: ClassTag, V]( partitions: Int, @transient rdd: RDD[_ <: Product2[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/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 9a4eefad2e..185ddb1fe5 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -1,71 +1,159 @@ package org.apache.spark -import scala.collection.JavaConversions._ -import scala.collection.concurrent.TrieMap +import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap import com.typesafe.config.ConfigFactory -private[spark] class SparkConf(loadClasspathRes: Boolean = true) extends Serializable { - @transient lazy val config = ConfigFactory.systemProperties() - .withFallback(ConfigFactory.parseResources("spark.conf")) - // TODO this should actually be synchronized - private val configMap = TrieMap[String, String]() +/** + * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. + * + * Most of the time, you would create a SparkConf object with `new SparkConf()`, which will load + * values from both the `spark.*` Java system properties and any `spark.conf` on your application's + * classpath (if it has one). In this case, system properties take priority over `spark.conf`, and + * any parameters you set directly on the `SparkConf` object take priority over both of those. + * + * For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and + * get the same configuration no matter what is on the classpath. + * + * @param loadDefaults whether to load values from the system properties and classpath + */ +class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { - if (loadClasspathRes && !config.entrySet().isEmpty) { - for (e <- config.entrySet()) { - configMap += ((e.getKey, e.getValue.unwrapped().toString)) + /** Create a SparkConf that loads defaults from system properties and the classpath */ + def this() = this(true) + + private val settings = new HashMap[String, String]() + + if (loadDefaults) { + val typesafeConfig = ConfigFactory.systemProperties() + .withFallback(ConfigFactory.parseResources("spark.conf")) + for (e <- typesafeConfig.entrySet().asScala) { + settings(e.getKey) = e.getValue.unwrapped.toString } } - def setMasterUrl(master: String) = { - if (master != null) - configMap += (("spark.master", master)) + /** Set a configuration variable. */ + def set(key: String, value: String): SparkConf = { + settings(key) = value + this + } + + /** + * The master URL to connect to, such as "local" to run locally with one thread, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + */ + def setMaster(master: String): SparkConf = { + if (master != null) { + settings("spark.master") = master + } this } - def setAppName(name: String) = { - if (name != null) - configMap += (("spark.appName", name)) + /** Set a name for your application. Shown in the Spark web UI. */ + def setAppName(name: String): SparkConf = { + if (name != null) { + settings("spark.appName") = name + } this } - def setJars(jars: Seq[String]) = { - if (!jars.isEmpty) - configMap += (("spark.jars", jars.mkString(","))) + /** Set JAR files to distribute to the cluster. */ + def setJars(jars: Seq[String]): SparkConf = { + if (!jars.isEmpty) { + settings("spark.jars") = jars.mkString(",") + } this } - def set(k: String, value: String) = { - configMap += ((k, value)) + /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ + def setJars(jars: Array[String]): SparkConf = { + if (!jars.isEmpty) { + settings("spark.jars") = jars.mkString(",") + } this } - def setSparkHome(home: String) = { - if (home != null) - configMap += (("spark.home", home)) + /** Set an environment variable to be used when launching executors for this application. */ + def setExecutorEnv(variable: String, value: String): SparkConf = { + settings("spark.executorEnv." + variable) = value this } - def set(map: Seq[(String, String)]) = { - if (map != null && !map.isEmpty) - configMap ++= map + /** Set multiple environment variables to be used when launching executors. */ + def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = { + for ((k, v) <- variables) { + setExecutorEnv(k, v) + } this } - def get(k: String): String = { - configMap(k) + /** + * Set multiple environment variables to be used when launching executors. + * (Java-friendly version.) + */ + def setExecutorEnv(variables: Array[(String, String)]): SparkConf = { + for ((k, v) <- variables) { + setExecutorEnv(k, v) + } + this } - def getAllConfiguration = configMap.clone.entrySet().iterator + /** + * Set the location where Spark is installed on worker nodes. This is only needed on Mesos if + * you are not using `spark.executor.uri` to disseminate the Spark binary distribution. + */ + def setSparkHome(home: String): SparkConf = { + if (home != null) { + settings("spark.home") = home + } + this + } + /** Set multiple parameters together */ + def setAll(settings: Traversable[(String, String)]) = { + this.settings ++= settings + this + } + + /** Set a parameter if it isn't already configured */ + def setIfMissing(key: String, value: String): SparkConf = { + if (!settings.contains(key)) { + settings(key) = value + } + this + } + + /** Get a parameter; throws an exception if it's not set */ + def get(key: String): String = { + settings(key) + } + + /** Get a parameter as an Option */ + def getOption(key: String): Option[String] = { + settings.get(key) + } + + /** Get all parameters as a list of pairs */ + def getAll: Seq[(String, String)] = settings.clone().toSeq + + /** Get a parameter, falling back to a default if not set */ def getOrElse(k: String, defaultValue: String): String = { - configMap.getOrElse(k, defaultValue) + settings.getOrElse(k, defaultValue) } - override def clone: SparkConf = { - val conf = new SparkConf(false) - conf.set(configMap.toSeq) - conf + /** Get all executor environment variables set on this SparkConf */ + def getExecutorEnv: Seq[(String, String)] = { + val prefix = "spark.executorEnv." + getAll.filter(pair => pair._1.startsWith(prefix)) + .map(pair => (pair._1.substring(prefix.length), pair._2)) } + /** Does the configuration contain a given parameter? */ + def contains(key: String): Boolean = settings.contains(key) + + /** Copy this object */ + override def clone: SparkConf = { + new SparkConf(false).setAll(settings) + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4300b07bdb..0567f7f437 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,8 +22,7 @@ import java.net.URI import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.{Map, immutable} -import scala.collection.JavaConversions._ +import scala.collection.{Map, Set, immutable} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -57,23 +56,32 @@ import org.apache.spark.util._ * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * - * @param conf a Spark Config object describing the context configuration. Any settings in this - * config overrides the default configs as well as system properties. - * - * @param environment Environment variables to set on worker nodes. + * @param conf_ a Spark Config object describing the application configuration. Any settings in + * this config overrides the default configs as well as system properties. + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. */ class SparkContext( - val conf: SparkConf, - val environment: Map[String, String] = Map(), + conf_ : SparkConf, // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) - // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set - // of data-local splits on host - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = immutable.Map()) + // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains + // a map from hostname to a list of input format splits on the host. + val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) extends Logging { /** - * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark - * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI + * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters + */ + def this(master: String, appName: String, conf: SparkConf) = + this(conf.setMaster(master).setAppName(appName)) + + /** + * Alternative constructor that allows setting common Spark properties directly * * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. @@ -82,24 +90,42 @@ class SparkContext( * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes. */ - def this(master: String, appName: String, sparkHome: String = null, - jars: Seq[String] = Nil, environment: Map[String, String] = Map(), - preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - immutable.Map()) = - this(new SparkConf(false).setAppName(appName).setMasterUrl(master) - .setJars(jars).set(environment.toSeq).setSparkHome(sparkHome), - environment, preferredNodeLocationData) + def this( + master: String, + appName: String, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map(), + preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = + { + this( + new SparkConf() + .setMaster(master) + .setAppName(appName) + .setJars(jars) + .setExecutorEnv(environment.toSeq) + .setSparkHome(sparkHome), + preferredNodeLocationData) + } - // Set Spark driver host and port system properties - Try(conf.get("spark.driver.host")) - .getOrElse(conf.set("spark.driver.host", Utils.localHostName())) + val conf = conf_.clone() + + if (!conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!conf.contains("spark.appName")) { + throw new SparkException("An application must be set in your configuration") + } - Try(conf.get("spark.driver.port")) - .getOrElse(conf.set("spark.driver.port", "0")) + // Set Spark driver host and port system properties + conf.setIfMissing("spark.driver.host", Utils.localHostName()) + conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) { - conf.get("spark.jars").split(",") - } else null + val jars: Seq[String] = if (conf.contains("spark.jars")) { + conf.get("spark.jars").split(",").filter(_.size != 0) + } else { + null + } val master = conf.get("spark.master") val appName = conf.get("spark.appName") @@ -115,8 +141,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, conf, - true, - isLocal) + isDriver = true, + isLocal = isLocal) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp @@ -125,7 +151,8 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup) + private[spark] val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) // Initialize the Spark UI private[spark] val ui = new SparkUI(this) @@ -135,9 +162,14 @@ class SparkContext( // Add each JAR given through the constructor if (jars != null) { - jars.foreach { addJar(_) } + jars.foreach(addJar) } + private[spark] val executorMemory = conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner @@ -148,10 +180,8 @@ class SparkContext( } } // Since memory can be set with a system property too, use that - executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m" - if (environment != null) { - executorEnvs ++= environment - } + executorEnvs("SPARK_MEM") = executorMemory + "m" + executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -183,12 +213,12 @@ class SparkContext( hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) } // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - Utils.getSystemProperties.foreach { case (key, value) => + conf.getAll.foreach { case (key, value) => if (key.startsWith("spark.hadoop.")) { hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + val bufferSize = conf.getOrElse("spark.buffer.size", "65536") hadoopConf.set("io.file.buffer.size", bufferSize) hadoopConf } @@ -200,7 +230,7 @@ class SparkContext( override protected def childValue(parent: Properties): Properties = new Properties(parent) } - private[spark] def getLocalProperties(): Properties = localProperties.get() + private[spark] def getLocalProperties: Properties = localProperties.get() private[spark] def setLocalProperties(props: Properties) { localProperties.set(props) @@ -533,7 +563,7 @@ class SparkContext( // Fetch the file locally in case a job is executed locally. // Jobs that run through LocalScheduler will already fetch the required dependencies, // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here. - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } @@ -915,14 +945,6 @@ object SparkContext { private[spark] val SPARK_UNKNOWN_USER = "" - private lazy val conf = new SparkConf() - - private[spark] def globalConf = { - if (SparkEnv.get != null) { - SparkEnv.get.conf - } else conf - } - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 @@ -1031,18 +1053,10 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) - /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ - private[spark] val executorMemoryRequested = { - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Try(globalConf.get("spark.executor.memory")).toOption - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - // Creates a task scheduler based on a given master URL. Extracted for testing. - private - def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = { + private def createTaskScheduler(sc: SparkContext, master: String, appName: String) + : TaskScheduler = + { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1076,10 +1090,10 @@ object SparkContext { case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt - if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { + if (sc.executorMemory > memoryPerSlaveInt) { throw new SparkException( "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( - memoryPerSlaveInt, SparkContext.executorMemoryRequested)) + memoryPerSlaveInt, sc.executorMemory)) } val scheduler = new ClusterScheduler(sc) @@ -1137,7 +1151,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(sc) - val coarseGrained = globalConf.getOrElse("spark.mesos.coarse", "false").toBoolean + val coarseGrained = sc.conf.getOrElse("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 78e4ae27b2..34fad3e763 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -107,7 +107,7 @@ object SparkEnv extends Logging { /** * Returns the ThreadLocal SparkEnv. */ - def getThreadLocal : SparkEnv = { + def getThreadLocal: SparkEnv = { env.get() } @@ -150,18 +150,19 @@ object SparkEnv extends Logging { val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) val closureSerializer = serializerManager.get( - conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), + conf) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") - val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt + val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") + val driverPort: Int = conf.getOrElse("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) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index acf328aa6a..e03cf9d13a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -29,17 +29,22 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import com.google.common.base.Optional -import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext} +import org.apache.spark._ import org.apache.spark.SparkContext.IntAccumulatorParam import org.apache.spark.SparkContext.DoubleAccumulatorParam import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import scala.Tuple2 /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and * works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { + /** + * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters + */ + def this(conf: SparkConf) = this(new SparkContext(conf)) /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). @@ -47,6 +52,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String) = this(new SparkContext(master, appName)) + /** + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI + * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters + */ + def this(master: String, appName: String, conf: SparkConf) = + this(conf.setMaster(master).setAppName(appName)) + /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index d6eacfe23e..05fd824254 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -247,10 +247,10 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[ */ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - import SparkContext.{globalConf => conf} + Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index cecb8c228b..47528bcee8 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - + def value = value_ def blockId = BroadcastBlockId(id) @@ -40,7 +40,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } - if (!isLocal) { + if (!isLocal) { HttpBroadcast.write(id, value_) } @@ -81,41 +81,48 @@ private object HttpBroadcast extends Logging { private var serverUri: String = null private var server: HttpServer = null + // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist private val files = new TimeStampedHashSet[String] - private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup) + private var cleaner: MetadataCleaner = null - private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt + private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt - private lazy val compressionCodec = CompressionCodec.createCodec() + private var compressionCodec: CompressionCodec = null def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt - compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean if (isDriver) { - createServer() + createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) } serverUri = conf.get("spark.httpBroadcast.uri") + cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup, conf) + compressionCodec = CompressionCodec.createCodec(conf) initialized = true } } } - + def stop() { synchronized { if (server != null) { server.stop() server = null } + if (cleaner != null) { + cleaner.cancel() + cleaner = null + } + compressionCodec = null initialized = false - cleaner.cancel() } } - private def createServer() { - broadcastDir = Utils.createTempDir(Utils.getLocalDir) + private def createServer(conf: SparkConf) { + broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) server = new HttpServer(broadcastDir) server.start() serverUri = server.uri @@ -143,7 +150,7 @@ private object HttpBroadcast extends Logging { val in = { val httpConnection = new URL(url).openConnection() httpConnection.setReadTimeout(httpReadTimeout) - val inputStream = httpConnection.getInputStream() + val inputStream = httpConnection.getInputStream if (compress) { compressionCodec.compressedInputStream(inputStream) } else { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 4a3801dc48..00ec3b971b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -83,13 +83,13 @@ extends Broadcast[T](id) with Logging with Serializable { case None => val start = System.nanoTime logInfo("Started reading broadcast variable " + id) - + // Initialize @transient variables that will receive garbage values from the master. resetWorkerVariables() if (receiveBroadcast(id)) { value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - + // Store the merged copy in cache so that the next worker doesn't need to rebuild it. // This creates a tradeoff between memory usage and latency. // Storing copy doubles the memory footprint; not storing doubles deserialization cost. @@ -122,14 +122,14 @@ extends Broadcast[T](id) with Logging with Serializable { while (attemptId > 0 && totalBlocks == -1) { TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => + case Some(x) => val tInfo = x.asInstanceOf[TorrentInfo] totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes arrayOfBlocks = new Array[TorrentBlock](totalBlocks) hasBlocks = 0 - - case None => + + case None => Thread.sleep(500) } } @@ -145,13 +145,13 @@ extends Broadcast[T](id) with Logging with Serializable { val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => + case Some(x) => arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] hasBlocks += 1 SparkEnv.get.blockManager.putSingle( pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true) - - case None => + + case None => throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) } } @@ -175,13 +175,13 @@ extends Logging { } } } - + def stop() { initialized = false } - lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 - + lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 + def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) val bais = new ByteArrayInputStream(byteArray) @@ -210,7 +210,7 @@ extends Logging { } def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, + totalBytes: Int, totalBlocks: Int): T = { var retByteArray = new Array[Byte](totalBytes) for (i <- 0 until totalBlocks) { @@ -223,22 +223,22 @@ extends Logging { } private[spark] case class TorrentBlock( - blockID: Int, - byteArray: Array[Byte]) + blockID: Int, + byteArray: Array[Byte]) extends Serializable private[spark] case class TorrentInfo( @transient arrayOfBlocks : Array[TorrentBlock], - totalBlocks: Int, - totalBytes: Int) + totalBlocks: Int, + totalBytes: Int) extends Serializable { - - @transient var hasBlocks = 0 + + @transient var hasBlocks = 0 } private[spark] class TorrentBroadcastFactory extends BroadcastFactory { - + def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index dda43dc018..19d393a0db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -26,7 +26,7 @@ private[spark] class ApplicationDescription( val appUiUrl: String) extends Serializable { - val user = System.getProperty("user.name", "") + val user = System.getProperty("user.name", "") override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 59d12a3e6f..ffc0cb0903 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -22,7 +22,7 @@ import akka.actor.ActorSystem import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master import org.apache.spark.util.Utils -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import scala.collection.mutable.ArrayBuffer @@ -43,7 +43,8 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") /* Start the Master */ - val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0) + val conf = new SparkConf(false) + val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf) masterActorSystems += masterSystem val masterUrl = "spark://" + localHostname + ":" + masterPort val masters = Array(masterUrl) @@ -55,7 +56,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I workerActorSystems += workerSystem } - return masters + masters } def stop() { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 1c979ac3e0..4f402c1121 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -34,10 +34,10 @@ class SparkHadoopUtil { UserGroupInformation.setConfiguration(conf) def runAsUser(user: String)(func: () => Unit) { - // if we are already running as the user intended there is no reason to do the doAs. It + // if we are already running as the user intended there is no reason to do the doAs. It // will actually break secure HDFS access as it doesn't fill in the credentials. Also if - // the user is UNKNOWN then we shouldn't be creating a remote unknown user - // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only + // the user is UNKNOWN then we shouldn't be creating a remote unknown user + // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only // in SparkContext. val currentUser = Option(System.getProperty("user.name")). getOrElse(SparkContext.SPARK_UNKNOWN_USER) @@ -67,12 +67,14 @@ class SparkHadoopUtil { } object SparkHadoopUtil { - import SparkContext.{globalConf => conf} + private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + val yarnMode = java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE")) if (yarnMode) { try { - Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] + Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") + .newInstance() + .asInstanceOf[SparkHadoopUtil] } catch { case th: Throwable => throw new SparkException("Unable to load YARN support", th) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 426cf524ae..ef649fd80c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { @@ -46,11 +46,12 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, - conf = SparkContext.globalConf) + conf = new SparkConf) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), + "dummy-spark-home", "ignored") val listener = new TestListener - val client = new Client(actorSystem, Array(url), desc, listener, SparkContext.globalConf) + val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf) client.start() actorSystem.awaitTermination() } 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 2c162c4fa2..9c89e36b14 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 @@ -29,7 +29,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{SparkContext, Logging, SparkException} +import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ @@ -38,14 +38,16 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { - import context.dispatcher - val conf = SparkContext.globalConf + import context.dispatcher // to use Akka's scheduler.schedule() + + val conf = new SparkConf + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -86,7 +88,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -495,7 +497,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } else { if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) - workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it } } } @@ -507,8 +509,9 @@ private[spark] object Master { val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { - val args = new MasterArguments(argStrings, SparkContext.globalConf) - val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + val conf = new SparkConf + val args = new MasterArguments(argStrings, conf) + val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf) actorSystem.awaitTermination() } @@ -522,11 +525,12 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, - conf = SparkContext.globalConf) + def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) + : (ActorSystem, Int, Int) = + { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) - val timeout = AkkaUtils.askTimeout(SparkContext.globalConf) + val timeout = AkkaUtils.askTimeout(conf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 79d95b1a83..60c7a7c2d6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging} */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, conf: SparkConf) extends Logging { - val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") + val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index df5bb368a2..a61597bbdf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() private val zk = new SparkZooKeeperSession(this, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index c55b720422..245a558a59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk = new SparkZooKeeperSession(this, conf) 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 75a6e75c78..f844fcbbfc 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 @@ -55,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -267,7 +267,7 @@ private[spark] class Worker( } private[spark] object Worker { - import org.apache.spark.SparkContext.globalConf + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, @@ -276,14 +276,16 @@ private[spark] object Worker { } def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, - masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) - : (ActorSystem, Int) = { + masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) + : (ActorSystem, Int) = + { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems + val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, - conf = globalConf) + conf = conf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir, globalConf), name = "Worker") + masterUrls, workDir, conf), name = "Worker") (actorSystem, boundPort) } 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 c8319f6f6e..53a2b94a52 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} @@ -98,7 +98,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = SparkContext.globalConf) + indestructible = true, conf = new SparkConf) // set it val sparkHostPort = hostname + ":" + boundPort // conf.set("spark.hostPort", sparkHostPort) 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 70fc30e993..a6eabc462b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -57,17 +57,18 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) + + // Set spark.* properties from executor arg val conf = new SparkConf(false) - // Set spark.* system properties from executor arg - for ((key, value) <- properties) { - conf.set(key, value) - } + conf.setAll(properties) // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. This will be used later when SparkEnv // created. - if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) { - conf.set("spark.local.dir", getYarnLocalDirs()) + if (java.lang.Boolean.valueOf( + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) + { + conf.set("spark.local.dir", getYarnLocalDirs()) } // Create our ClassLoader and set it on this thread @@ -331,12 +332,12 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 8ef5019b6c..20402686a8 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,7 +22,7 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkEnv, SparkConf} /** @@ -38,16 +38,15 @@ trait CompressionCodec { private[spark] object CompressionCodec { - import org.apache.spark.SparkContext.globalConf - def createCodec(): CompressionCodec = { - createCodec(System.getProperty( + def createCodec(conf: SparkConf): CompressionCodec = { + createCodec(conf, conf.getOrElse( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } - def createCodec(codecName: String): CompressionCodec = { + def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) .getConstructor(classOf[SparkConf]) - ctor.newInstance(globalConf).asInstanceOf[CompressionCodec] + ctor.newInstance(conf).asInstanceOf[CompressionCodec] } } @@ -72,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 3e902f8ac5..697096fa76 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -593,10 +593,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private[spark] object ConnectionManager { - import SparkContext.globalConf - def main(args: Array[String]) { - val manager = new ConnectionManager(9999, globalConf) + val manager = new ConnectionManager(9999, new SparkConf) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 4ca3cd390b..1c9d6030d6 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -19,19 +19,19 @@ package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress +import org.apache.spark.SparkConf private[spark] object ReceiverTest { - import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - val manager = new ConnectionManager(9999, globalConf) + val manager = new ConnectionManager(9999, new SparkConf) println("Started connection manager with id = " + manager.id) - - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ - val buffer = ByteBuffer.wrap("response".getBytes()) + val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) - Thread.currentThread.join() + Thread.currentThread.join() } } diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 11c21fc1d5..dcbd183c88 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -19,29 +19,29 @@ package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress +import org.apache.spark.SparkConf private[spark] object SenderTest { - import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - + if (args.length < 2) { println("Usage: SenderTest ") System.exit(1) } - + val targetHost = args(0) val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val manager = new ConnectionManager(0, globalConf) + val manager = new ConnectionManager(0, new SparkConf) println("Started connection manager with id = " + manager.id) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None }) - - val size = 100 * 1024 * 1024 + + val size = 100 * 1024 * 1024 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -50,7 +50,7 @@ private[spark] object SenderTest { val count = 100 (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) - val startTime = System.currentTimeMillis + val startTime = System.currentTimeMillis /*println("Started timer at " + startTime)*/ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match { case Some(response) => diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index 81b3104afd..db28ddf9ac 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { @@ -104,10 +104,10 @@ private[spark] object ShuffleCopier extends Logging { val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - val tasks = (for (i <- Range(0, threads)) yield { + val tasks = (for (i <- Range(0, threads)) yield { Executors.callable(new Runnable() { def run() { - val copier = new ShuffleCopier(SparkContext.globalConf) + val copier = new ShuffleCopier(new SparkConf) copier.getBlock(host, port, blockId, echoResultCollectCallBack) } }) 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 9fbe002748..2897c4b841 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -74,9 +74,6 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) } private[spark] object CheckpointRDD extends Logging { - - import SparkContext.{globalConf => conf} - def splitIdToFile(splitId: Int): String = { "part-%05d".format(splitId) } @@ -94,7 +91,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -124,7 +121,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 911a002884..4ba4696fef 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: map.changeValue(k, update) } - val ser = SparkEnv.get.serializerManager.get(serializerClass) + val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent 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 3682c84598..0ccb309d0d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -59,7 +59,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[P] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, - SparkEnv.get.serializerManager.get(serializerClass)) + SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index aab30b1bb4..4f90c7d3d6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -93,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = SparkEnv.get.serializerManager.get(serializerClass) + val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) 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 963d15b76d..77aa24e6b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -158,7 +158,8 @@ class DAGScheduler( val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup) + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) /** * Starts the event processing actor. The actor has two responsibilities: @@ -529,7 +530,7 @@ class DAGScheduler( case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => var finalStage: Stage = null try { - // New stage creation at times and if its not protected, the scheduler thread is killed. + // New stage creation at times and if its not protected, the scheduler thread is killed. // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 1791ee660d..90eb8a747f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -32,7 +32,7 @@ import scala.collection.JavaConversions._ /** * Parses and holds information about inputFormat (and files) specified as a parameter. */ -class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], +class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { var mapreduceInputFormat: Boolean = false @@ -40,7 +40,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl validate() - override def toString(): String = { + override def toString: String = { "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path } @@ -125,7 +125,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } private def findPreferredLocations(): Set[SplitInfo] = { - logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + + logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + ", inputFormatClazz : " + inputFormatClazz) if (mapreduceInputFormat) { return prefLocsFromMapreduceInputFormat() @@ -143,14 +143,14 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl object InputFormatInfo { /** Computes the preferred locations based on input(s) and returned a location to block map. - Typical use of this method for allocation would follow some algo like this - (which is what we currently do in YARN branch) : + Typical use of this method for allocation would follow some algo like this: + a) For each host, count number of splits hosted on that host. b) Decrement the currently allocated containers on that host. c) Compute rack info for each host and update rack -> count map based on (b). d) Allocate nodes based on (c) - e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node - (even if data locality on that is very high) : this is to prevent fragility of job if a single + e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node + (even if data locality on that is very high) : this is to prevent fragility of job if a single (or small set of) hosts go down. go to (a) until required nodes are allocated. diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 3f55cd5642..60927831a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - def this() = this(System.getProperty("user.name", ""), + def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) private val logDir = diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 310ec62ca8..28f3ba53b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -32,7 +32,9 @@ private[spark] object ResultTask { // expensive on the master node if it needs to launch thousands of tasks. val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues) + // TODO: This object shouldn't have global variables + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 9002d33cda..3cf995ea74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -52,7 +52,7 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file")) + val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file") val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 0f2deb4bcb..a37ead5632 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -37,7 +37,9 @@ private[spark] object ShuffleMapTask { // expensive on the master node if it needs to launch thousands of tasks. val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues) + // TODO: This object shouldn't have global variables + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf) def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { synchronized { @@ -152,7 +154,7 @@ private[spark] class ShuffleMapTask( try { // Obtain all the block writers for shuffle blocks. - val ser = SparkEnv.get.serializerManager.get(dep.serializerClass) + val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf) shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) // Write the map output to its associated buckets. 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 7e231ec44c..2707740d44 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 @@ -51,10 +51,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext) { val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -91,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -120,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (conf.getOrElse("spark.speculation", "false").toBoolean) { + if (conf.getOrElse("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, 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 398b0cefbf..a46b16b92f 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 @@ -52,14 +52,14 @@ private[spark] class ClusterTaskSetManager( { val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt + val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -118,7 +118,7 @@ private[spark] class ClusterTaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception @@ -678,7 +678,7 @@ private[spark] class ClusterTaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.getOrElse("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => conf.getOrElse("spark.locality.wait.process", defaultWait).toLong diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 40555903ac..156b01b149 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -62,7 +62,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -118,7 +118,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac removeExecutor(executorId, reason) sender ! true - case DisassociatedEvent(_, address, _) => + case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) } @@ -163,10 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac override def start() { val properties = new ArrayBuffer[(String, String)] - val iterator = scheduler.sc.conf.getAllConfiguration - while (iterator.hasNext) { - val entry = iterator.next - val (key, value) = (entry.getKey.toString, entry.getValue.toString) + for ((key, value) <- scheduler.sc.conf.getAll) { if (key.startsWith("spark.") && !key.equals("spark.hostPort")) { properties += ((key, value)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala index 5367218faa..65d3fc8187 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala @@ -31,7 +31,4 @@ private[spark] trait SchedulerBackend { def defaultParallelism(): Int def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException - - // Memory used by each executor (in megabytes) - protected val executorMemory: Int = SparkContext.executorMemoryRequested } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d01329b2b3..d74f000ebb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -31,7 +31,7 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt override def start() { super.start() 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 d6b8ac2d57..de69e3260d 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 @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) - val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, + val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome, "http://" + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, masters, appDesc, this, conf) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index ff6cc37f1d..319c91b933 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt + private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") 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 2a3b0e15f7..1695374152 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 @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 @@ -176,7 +176,7 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveId = offer.getSlaveId.toString val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 && + if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave @@ -192,7 +192,7 @@ private[spark] class CoarseMesosSchedulerBackend( .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", executorMemory)) + .addResources(createResource("mem", sc.executorMemory)) .build() d.launchTasks(offer.getId, Collections.singletonList(task), filters) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 9bb92b4f01..8dfd4d5fb3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -114,7 +114,7 @@ private[spark] class MesosSchedulerBackend( val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build()) + .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build()) .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -199,7 +199,7 @@ private[spark] class MesosSchedulerBackend( def enoughMemory(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") val slaveId = o.getSlaveId.getValue - mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) + mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId) } for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { @@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index 6069c1db3a..8498cffd31 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -92,7 +92,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 4de81617b1..5d3d43623d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.SparkConf private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { val objOut = new ObjectOutputStream(out) @@ -77,6 +78,6 @@ private[spark] class JavaSerializerInstance extends SerializerInstance { /** * A Spark serializer that uses Java's built-in serialization. */ -class JavaSerializer extends Serializer { +class JavaSerializer(conf: SparkConf) extends Serializer { def newInstance(): SerializerInstance = new JavaSerializerInstance } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 17cec81038..2367f3f521 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -25,20 +25,21 @@ import com.esotericsoftware.kryo.{KryoException, Kryo} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} -import org.apache.spark.{SparkContext, SparkConf, SerializableWritable, Logging} +import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import scala.util.Try +import org.apache.spark.storage.PutBlock +import org.apache.spark.storage.GetBlock +import org.apache.spark.storage.GotBlock /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ -class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { - - private val conf = SparkContext.globalConf +class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = { - conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -50,7 +51,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 2955986fec..22465272f3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.SparkConf /** @@ -32,12 +33,12 @@ private[spark] class SerializerManager { def default = _default - def setDefault(clsName: String): Serializer = { - _default = get(clsName) + def setDefault(clsName: String, conf: SparkConf): Serializer = { + _default = get(clsName, conf) _default } - def get(clsName: String): Serializer = { + def get(clsName: String, conf: SparkConf): Serializer = { if (clsName == null) { default } else { @@ -51,8 +52,9 @@ private[spark] class SerializerManager { serializer = serializers.get(clsName) if (serializer == null) { val clsLoader = Thread.currentThread.getContextClassLoader - serializer = - Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + val cls = Class.forName(clsName, true, clsLoader) + val constructor = cls.getConstructor(classOf[SparkConf]) + serializer = constructor.newInstance(conf).asInstanceOf[Serializer] serializers.put(clsName, serializer) } serializer diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index ee2ae471a9..3b25f68ca8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index ffd166e93a..16ee208617 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -58,8 +58,8 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } @@ -72,18 +72,18 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean - val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties + val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - val hostPort = Utils.localHostPort() + val hostPort = Utils.localHostPort(conf) val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) @@ -101,8 +101,11 @@ private[spark] class BlockManager( var heartBeatTask: Cancellable = null - private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks) - private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks) + private val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) + private val broadcastCleaner = new MetadataCleaner( + MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) + initialize() // The compression codec to use. Note that the "lazy" val is necessary because we want to delay @@ -110,14 +113,14 @@ private[spark] class BlockManager( // program could be using a user-defined codec in a third party jar, which is loaded in // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been // loaded yet. - private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec() + private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) /** * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, serializer: Serializer, conf: SparkConf) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf) + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf) } /** @@ -127,7 +130,7 @@ private[spark] class BlockManager( private def initialize() { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) - if (!BlockManager.getDisableHeartBeatsForTesting) { + if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { heartBeat() } @@ -440,7 +443,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -466,7 +469,7 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } @@ -858,19 +861,18 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - import org.apache.spark.SparkContext.{globalConf => conf} val ID_GENERATOR = new IdGenerator - def getMaxMemoryFromSystemProperties: Long = { - val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble + def getMaxMemory(conf: SparkConf): Long = { + val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } - def getHeartBeatFrequencyFromSystemProperties: Long = - conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + def getHeartBeatFrequency(conf: SparkConf): Long = + conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 - def getDisableHeartBeatsForTesting: Boolean = - conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = + conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that 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 fde7d63a68..8e4a88b20a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -31,8 +31,8 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 05502e4451..73a1da2de6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -53,7 +53,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act initLogging() val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", - "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong + "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong @@ -61,7 +61,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act var timeoutCheckingTask: Cancellable = null override def preStart() { - if (!BlockManager.getDisableHeartBeatsForTesting) { + if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule( 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 8f528babd4..7697092e1b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 850d3178dd..f592df283a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -62,12 +62,13 @@ private[spark] trait ShuffleWriterGroup { private[spark] class ShuffleBlockManager(blockManager: BlockManager) { def conf = blockManager.conf + // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused @@ -82,8 +83,8 @@ class ShuffleBlockManager(blockManager: BlockManager) { type ShuffleId = Int private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] - private - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup) + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { new ShuffleWriterGroup { diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index d52b3d8284..40734aab49 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -56,7 +56,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong) = { val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer()) + new KryoSerializer(sc.conf)) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeData) 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 b3b3893393..dca98c6c05 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,7 +22,7 @@ import akka.actor._ import java.util.concurrent.ArrayBlockingQueue import util.Random import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -92,8 +92,8 @@ private[spark] object ThreadingTest { def main(args: Array[String]) { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") - val conf = SparkContext.globalConf - val serializer = new KryoSerializer + val conf = new SparkConf() + val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) val blockManager = new BlockManager( diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 14751e8e8e..58d47a201d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui import scala.util.Random -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.SchedulingMode @@ -31,7 +31,6 @@ import org.apache.spark.scheduler.SchedulingMode */ private[spark] object UIWorkloadGenerator { - import SparkContext.{globalConf => conf} val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 @@ -40,14 +39,14 @@ private[spark] object UIWorkloadGenerator { println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } - val master = args(0) - val schedulingMode = SchedulingMode.withName(args(1)) - val appName = "Spark UI Tester" + val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester") + + val schedulingMode = SchedulingMode.withName(args(1)) if (schedulingMode == SchedulingMode.FAIR) { - conf.set("spark.scheduler.mode", "FAIR") + conf.set("spark.scheduler.mode", "FAIR") } - val sc = new SparkContext(master, appName) + val sc = new SparkContext(conf) def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { @@ -57,11 +56,11 @@ private[spark] object UIWorkloadGenerator { } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) - def nextFloat() = (new Random()).nextFloat() + def nextFloat() = new Random().nextFloat() val jobs = Seq[(String, () => Long)]( ("Count", baseData.count), - ("Cache and Count", baseData.map(x => x).cache.count), + ("Cache and Count", baseData.map(x => x).cache().count), ("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count), ("Entirely failed phase", baseData.map(x => throw new Exception).count), ("Partially failed phase", { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index b637d37517..91fa00a66c 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -63,7 +63,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(sc.conf.getOrElse("path.separator", ":")) + .split(sc.conf.getOrElse("path.separator", ":")) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index f01a1380b9..6ff8e9fb14 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() @@ -105,7 +105,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) stages += stage } - + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val sid = taskStart.task.stageId val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) 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 76febd5702..58b26f7f12 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -41,19 +41,19 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt - val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt - val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt - val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt + conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -89,6 +89,6 @@ private[spark] object AkkaUtils { /** Returns the default Spark timeout to use for Akka ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") + Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index bf71d17a21..431d88838f 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -18,16 +18,21 @@ package org.apache.spark.util import java.util.{TimerTask, Timer} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} /** * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) */ -class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging { +class MetadataCleaner( + cleanerType: MetadataCleanerType.MetadataCleanerType, + cleanupFunc: (Long) => Unit, + conf: SparkConf) + extends Logging +{ val name = cleanerType.toString - private val delaySeconds = MetadataCleaner.getDelaySeconds + private val delaySeconds = MetadataCleaner.getDelaySeconds(conf) private val periodSeconds = math.max(10, delaySeconds / 10) private val timer = new Timer(name + " cleanup timer", true) @@ -65,22 +70,28 @@ object MetadataCleanerType extends Enumeration { def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString } +// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the +// initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { - private val conf = SparkContext.globalConf - // using only sys props for now : so that workers can also get to it while preserving earlier behavior. - def getDelaySeconds = conf.getOrElse("spark.cleaner.ttl", "3500").toInt //TODO: this is to fix tests for time being + def getDelaySeconds(conf: SparkConf) = { + conf.getOrElse("spark.cleaner.ttl", "3500").toInt + } - def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt + def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = + { + conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) + .toInt } - def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { + def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType, + delay: Int) + { conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } - def setDelaySeconds(delay: Int, resetAll: Boolean = true) { + def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) { // override for all ? - conf.set("spark.cleaner.ttl", delay.toString) + conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 1407c39bfb..bddb3bb735 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -30,10 +30,10 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging} /** - * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in + * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in * memory-aware caches. * * Based on the following JavaWorld article: @@ -41,7 +41,6 @@ import org.apache.spark.{SparkConf, SparkContext, Logging} */ private[spark] object SizeEstimator extends Logging { - private def conf = SparkContext.globalConf // Sizes of primitive types private val BYTE_SIZE = 1 private val BOOLEAN_SIZE = 1 @@ -90,9 +89,11 @@ private[spark] object SizeEstimator extends Logging { classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil)) } - private def getIsCompressedOops : Boolean = { - if (conf.getOrElse("spark.test.useCompressedOops", null) != null) { - return conf.get("spark.test.useCompressedOops").toBoolean + private def getIsCompressedOops: Boolean = { + // This is only used by tests to override the detection of compressed oops. The test + // actually uses a system property instead of a SparkConf, so we'll stick with that. + if (System.getProperty("spark.test.useCompressedOops") != null) { + return System.getProperty("spark.test.useCompressedOops").toBoolean } try { @@ -104,7 +105,7 @@ private[spark] object SizeEstimator extends Logging { val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", Class.forName("java.lang.String")) - val bean = ManagementFactory.newPlatformMXBeanProxy(server, + val bean = ManagementFactory.newPlatformMXBeanProxy(server, hotSpotMBeanName, hotSpotMBeanClass) // TODO: We could use reflection on the VMOption returned ? return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") @@ -252,7 +253,7 @@ private[spark] object SizeEstimator extends Logging { if (info != null) { return info } - + val parent = getClassInfo(cls.getSuperclass) var shellSize = parent.shellSize var pointerFields = parent.pointerFields 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 fd5888e525..b6b89cc7bb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -36,15 +36,13 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer -import org.apache.spark.{SparkContext, SparkException, Logging} +import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging} /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { - - private lazy val conf = SparkContext.globalConf /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -240,9 +238,9 @@ private[spark] object Utils extends Logging { * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - def fetchFile(url: String, targetDir: File) { + def fetchFile(url: String, targetDir: File, conf: SparkConf) { val filename = url.split("/").last - val tempDir = getLocalDir + val tempDir = getLocalDir(conf) val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) val uri = new URI(url) @@ -312,7 +310,7 @@ private[spark] object Utils extends Logging { * return a single directory, even though the spark.local.dir property might be a list of * multiple paths. */ - def getLocalDir: String = { + def getLocalDir(conf: SparkConf): String = { conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } @@ -398,7 +396,7 @@ private[spark] object Utils extends Logging { InetAddress.getByName(address).getHostName } - def localHostPort(): String = { + def localHostPort(conf: SparkConf): String = { val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") @@ -838,7 +836,7 @@ private[spark] object Utils extends Logging { } } - /** + /** * Timing method based on iterations that permit JVM JIT optimization. * @param numIters number of iterations * @param f function to be executed diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index ab81bfbe55..8d7546085f 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -20,9 +20,11 @@ package org.apache.spark.io import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import org.scalatest.FunSuite +import org.apache.spark.SparkConf class CompressionCodecSuite extends FunSuite { + val conf = new SparkConf(false) def testCodec(codec: CompressionCodec) { // Write 1000 integers to the output stream, compressed. @@ -43,19 +45,19 @@ class CompressionCodecSuite extends FunSuite { } test("default compression codec") { - val codec = CompressionCodec.createCodec() + val codec = CompressionCodec.createCodec(conf) assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } test("lzf compression codec") { - val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName) + val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName) assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } test("snappy compression codec") { - val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName) + val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName) assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index 2bb827c022..3711382f2e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -82,7 +82,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} private val conf = new SparkConf - val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong + val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong test("TaskSet with no preferences") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index c016c51171..33b0148896 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -22,12 +22,14 @@ import scala.collection.mutable import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.SharedSparkContext +import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { + val conf = new SparkConf(false) + test("basic types") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -57,7 +59,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("pairs") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -81,7 +83,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("Scala data structures") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -104,7 +106,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("ranges") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time @@ -127,7 +129,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("custom registrator") { System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } 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 4ef5538951..a0fc3445be 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.{SparkConf, SparkContext} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { - private val conf = new SparkConf + private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -45,7 +45,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") - val serializer = new KryoSerializer + val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) @@ -167,7 +167,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("master + 2 managers interaction") { store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -654,7 +654,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200, conf) + store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index a5facd5bbd..11ebdc352b 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -140,8 +140,6 @@ class SizeEstimatorSuite test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") - SparkContext.globalConf.set("os.arch", "amd64") - SparkContext.globalConf.set("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 12c430be27..4c0de46964 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -37,7 +37,7 @@ object WikipediaPageRank { System.exit(-1) } val sparkConf = new SparkConf() - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) @@ -46,7 +46,7 @@ object WikipediaPageRank { val host = args(3) val usePartitioner = args(4).toBoolean - sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + sparkConf.setMaster(host).setAppName("WikipediaPageRank") val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 5bf0b7a24a..2cf273a702 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -35,7 +35,7 @@ object WikipediaPageRankStandalone { System.exit(-1) } val sparkConf = new SparkConf() - sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) @@ -44,7 +44,7 @@ object WikipediaPageRankStandalone { val host = args(3) val usePartitioner = args(4).toBoolean - sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone") val sc = new SparkContext(sparkConf) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 2f2d106f86..8b27ecf82c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -579,12 +579,12 @@ object ALS { val alpha = if (args.length >= 8) args(7).toDouble else 1 val blocks = if (args.length == 9) args(8).toInt else -1 val sc = new SparkContext(master, "ALS") - sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName) - sc.conf.set("spark.kryo.referenceTracking", "false") - sc.conf.set("spark.kryoserializer.buffer.mb", "8") - sc.conf.set("spark.locality.wait", "10000") - + sc.conf.set("spark.kryo.referenceTracking", "false") + sc.conf.set("spark.kryoserializer.buffer.mb", "8") + sc.conf.set("spark.locality.wait", "10000") + val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 433268a1dd..91e35e2d34 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) - + private var rpc: YarnRPC = YarnRPC.create(conf) private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var appAttemptId: ApplicationAttemptId = _ @@ -81,12 +81,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Workaround until hadoop moves to something which has // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) - + ApplicationMaster.register(this) // Start the user's JAR userThread = startUserClass() - + // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. waitForSparkMaster() @@ -99,7 +99,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Allocate all containers allocateWorkers() - // Wait for the user class to Finish + // Wait for the user class to Finish userThread.join() System.exit(0) @@ -119,7 +119,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } localDirs } - + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) @@ -128,17 +128,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("ApplicationAttemptId: " + appAttemptId) appAttemptId } - + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) } - + private def waitForSparkMaster() { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while (!driverUp && tries < numTries) { val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") @@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -214,7 +214,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, amClient, appAttemptId, - args, + args, sparkContext.preferredNodeLocationData) } else { logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". @@ -265,7 +265,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -314,11 +314,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e for (container <- containers) { logInfo("Launching shell command on a new container." + ", containerId=" + container.getId() - + ", containerNode=" + container.getNodeId().getHost() + + ", containerNode=" + container.getNodeId().getHost() + ":" + container.getNodeId().getPort() + ", containerNodeURI=" + container.getNodeHttpAddress() + ", containerState" + container.getState() - + ", containerResourceMemory" + + ", containerResourceMemory" + container.getResource().getMemory()) } } @@ -338,12 +338,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } /** - * Clean up the staging directory. + * Clean up the staging directory. */ - private def cleanupStagingDir() { + private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -359,7 +359,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { def run() { @@ -415,18 +415,18 @@ object ApplicationMaster { // Note that this will unfortunately not properly clean up the staging files because it gets // called too late, after the filesystem is already shutdown. if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } - } + } } ) } diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a322f60864..963b5b88be 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -150,7 +150,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl queueInfo.getChildQueues.size)) } - def verifyClusterResources(app: GetNewApplicationResponse) = { + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) @@ -221,7 +221,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) - } + } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) @@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -269,7 +269,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } } @@ -283,7 +283,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val destPath = copyRemoteFile(dst, localPath, replication) // Only add the resource to the Spark ApplicationMaster. val appMasterOnly = true - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache, appMasterOnly) } } @@ -295,7 +295,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache) } } @@ -307,7 +307,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, linkname, statCache) } } @@ -317,7 +317,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def setupLaunchEnv( - localResources: HashMap[String, LocalResource], + localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) @@ -406,11 +406,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val commands = List[String]( - javaCommand + + javaCommand + " -server " + JAVA_OPTS + " " + args.amClass + - " --class " + args.userClass + + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + " --worker-memory " + args.workerMemory + @@ -436,7 +436,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl super.submitApplication(appContext) } - def monitorApplication(appId: ApplicationId): Boolean = { + def monitorApplication(appId: ApplicationId): Boolean = { while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -458,7 +458,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() - if (state == YarnApplicationState.FINISHED || + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { return true @@ -495,25 +495,25 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + SPARK_JAR) Client.populateHadoopClasspath(conf, env) if (!userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + "*") } } diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 41ac292249..1a9bb97b3e 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,7 +35,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" 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 b2f499e637..f108c70f21 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -35,6 +35,7 @@ import java.lang.{Class => jClass} import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging +import org.apache.spark.SparkConf import org.apache.spark.SparkContext /** The Scala interactive shell. It provides a read-eval-print loop @@ -929,7 +930,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } def createSparkContext(): SparkContext = { - val uri = System.getenv("SPARK_EXECUTOR_URI") + val execUri = System.getenv("SPARK_EXECUTOR_URI") val master = this.master match { case Some(m) => m case None => { @@ -938,11 +939,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) - sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) - if (uri != null) { - sparkContext.conf.set("spark.executor.uri", uri) + val conf = new SparkConf() + .setMaster(master) + .setAppName("Spark shell") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(jars) + .set("spark.repl.class.uri", intp.classServer.uri) + if (execUri != null) { + conf.set("spark.executor.uri", execUri) } - sparkContext.conf.set("spark.repl.class.uri", intp.classServer.uri) + sparkContext = new SparkContext(conf) echo("Created spark context..") sparkContext } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 0d412e4478..a993083164 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -34,7 +34,7 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging} +import org.apache.spark.{HttpServer, SparkConf, Logging} import org.apache.spark.util.Utils // /** directory to save .class files to */ @@ -89,7 +89,7 @@ import org.apache.spark.util.Utils /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir", tmp) + val rootDir = new SparkConf().getOrElse("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index b8e1427a21..f106bba678 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -24,7 +24,7 @@ import java.util.concurrent.RejectedExecutionException import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.MetadataCleaner @@ -36,12 +36,11 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val framework = ssc.sc.appName val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars - val environment = ssc.sc.environment val graph = ssc.graph val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() - val delaySeconds = MetadataCleaner.getDelaySeconds + val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConf = ssc.sc.conf def validate() { @@ -58,7 +57,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) * Convenience class to speed up the writing of graph checkpoint to file */ private[streaming] -class CheckpointWriter(checkpointDir: String) extends Logging { +class CheckpointWriter(conf: SparkConf, checkpointDir: String) extends Logging { val file = new Path(checkpointDir, "graph") // The file to which we actually write - and then "move" to file. private val writeFile = new Path(file.getParent, file.getName + ".next") @@ -66,14 +65,14 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private var stopped = false - val conf = new Configuration() - var fs = file.getFileSystem(conf) + val hadoopConf = new Configuration() + var fs = file.getFileSystem(hadoopConf) val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) - private val compressionCodec = CompressionCodec.createCodec() + private val compressionCodec = CompressionCodec.createCodec(conf) - // Removed code which validates whether there is only one CheckpointWriter per path 'file' since + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since // I did not notice any errors - reintroduce it ? class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { @@ -142,11 +141,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private[streaming] object CheckpointReader extends Logging { - def read(path: String): Checkpoint = { + def read(conf: SparkConf, path: String): Checkpoint = { val fs = new Path(path).getFileSystem(new Configuration()) - val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) + val attempts = Seq( + new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) - val compressionCodec = CompressionCodec.createCodec() + val compressionCodec = CompressionCodec.createCodec(conf) attempts.foreach(file => { if (fs.exists(file)) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 329d2b5835..8005202500 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -213,7 +213,7 @@ abstract class DStream[T: ClassTag] ( checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." ) - val metadataCleanerDelay = MetadataCleaner.getDelaySeconds + val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf) logInfo("metadataCleanupDelay = " + metadataCleanerDelay) assert( metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index 1d23713c80..82ed6bed69 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -26,10 +26,10 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt + val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(ssc.checkpointDir) + new CheckpointWriter(ssc.conf, ssc.checkpointDir) } else { null } @@ -50,13 +50,13 @@ class Scheduler(ssc: StreamingContext) extends Logging { } logInfo("Scheduler started") } - + def stop() = synchronized { timer.stop() jobManager.stop() if (checkpointWriter != null) checkpointWriter.stop() ssc.graph.stop() - logInfo("Scheduler stopped") + logInfo("Scheduler stopped") } private def startFirstTime() { @@ -73,7 +73,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } 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 76744223e1..079841ad9d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -87,13 +87,12 @@ class StreamingContext private ( null, batchDuration) } - /** * Re-create a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or * to the checkpoint file 'graph' or 'graph.bk'. */ - def this(path: String) = this(null, CheckpointReader.read(path), null) + def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null) initLogging() @@ -102,11 +101,13 @@ class StreamingContext private ( "both SparkContext and checkpoint as null") } - if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) { - MetadataCleaner.setDelaySeconds(cp_.delaySeconds) + private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf) + + if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) { + MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds) } - if (MetadataCleaner.getDelaySeconds < 0) { + if (MetadataCleaner.getDelaySeconds(conf_) < 0) { throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") } @@ -115,12 +116,14 @@ class StreamingContext private ( protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.sparkConf, cp_.environment) + new SparkContext(cp_.sparkConf) } else { sc_ } } + protected[streaming] val conf = sc.conf + protected[streaming] val env = SparkEnv.get protected[streaming] val graph: DStreamGraph = { @@ -579,13 +582,15 @@ object StreamingContext { appName: String, sparkHome: String, jars: Seq[String], - environment: Map[String, String]): SparkContext = { + environment: Map[String, String]): SparkContext = + { + val sc = new SparkContext(master, appName, sparkHome, jars, environment) // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. - if (MetadataCleaner.getDelaySeconds < 0) { - MetadataCleaner.setDelaySeconds(3600) + if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) { + MetadataCleaner.setDelaySeconds(sc.conf, 3600) } - new SparkContext(master, appName, sparkHome, jars, environment) + sc } protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = { 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 8bf761b8cb..bd607f9d18 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 @@ -175,8 +175,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = env.conf.getOrElse("spark.driver.host", "localhost") - val port = env.conf.getOrElse("spark.driver.port", "7077").toInt + val ip = env.conf.getOrElse("spark.driver.host", "localhost") + val port = env.conf.getOrElse("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -213,7 +213,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index fc8655a083..6585d494a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import org.apache.spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket -import org.apache.spark.{Logging} +import org.apache.spark.{SparkConf, Logging} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException @@ -42,7 +42,7 @@ object RawTextSender extends Logging { // Repeat the input data multiple times to fill in a buffer val lines = Source.fromFile(file).getLines().toArray val bufferStream = new FastByteArrayOutputStream(blockSize + 1000) - val ser = new KryoSerializer().newInstance() + val ser = new KryoSerializer(new SparkConf()).newInstance() val serStream = ser.serializeStream(bufferStream) var i = 0 while (bufferStream.position < blockSize) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index da8f135dd7..8c16daa21c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -52,9 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" - before { - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") - } + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown @@ -70,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) ssc.registerOutputStream(outputStream) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index d1cab0c609..a265284bff 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -130,7 +130,11 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Whether to actually wait in real time before changing manual clock def actuallyWait = false - def conf = new SparkConf().setMasterUrl(master).setAppName(framework).set("spark.cleaner.ttl", "3600") + val conf = new SparkConf() + .setMaster(master) + .setAppName(framework) + .set("spark.cleaner.ttl", "3600") + /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 1dd38dd13e..dc9228180f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.Utils class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) - + private var rpc: YarnRPC = YarnRPC.create(conf) private var resourceManager: AMRMProtocol = _ private var appAttemptId: ApplicationAttemptId = _ @@ -68,7 +68,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) - + appAttemptId = getApplicationAttemptId() isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts resourceManager = registerWithResourceManager() @@ -92,11 +92,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // } //} // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) - + ApplicationMaster.register(this) // Start the user's JAR userThread = startUserClass() - + // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. waitForSparkMaster() @@ -105,11 +105,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Do this after spark master is up and SparkContext is created so that we can register UI Url val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - + // Allocate all containers allocateWorkers() - - // Wait for the user class to Finish + + // Wait for the user class to Finish userThread.join() System.exit(0) @@ -129,7 +129,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } localDirs } - + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) @@ -138,7 +138,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("ApplicationAttemptId: " + appAttemptId) appAttemptId } - + private def registerWithResourceManager(): AMRMProtocol = { val rmAddress = NetUtils.createSocketAddr(yarnConf.get( YarnConfiguration.RM_SCHEDULER_ADDRESS, @@ -146,26 +146,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Connecting to ResourceManager at " + rmAddress) rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] } - + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) .asInstanceOf[RegisterApplicationMasterRequest] appMasterRequest.setApplicationAttemptId(appAttemptId) // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. + // sensible info. // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) appMasterRequest.setTrackingUrl(uiAddress) resourceManager.registerApplicationMaster(appMasterRequest) } - + private def waitForSparkMaster() { logInfo("Waiting for spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while(!driverUp && tries < numTries) { val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") @@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -241,8 +241,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, resourceManager, appAttemptId, - args, - sparkContext.preferredNodeLocationData) + args, + sparkContext.preferredNodeLocationData) } else { logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". format(count * waitTime, numTries)) @@ -294,7 +294,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -342,11 +342,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e for (container <- containers) { logInfo("Launching shell command on a new container." + ", containerId=" + container.getId() - + ", containerNode=" + container.getNodeId().getHost() + + ", containerNode=" + container.getNodeId().getHost() + ":" + container.getNodeId().getPort() + ", containerNodeURI=" + container.getNodeHttpAddress() + ", containerState" + container.getState() - + ", containerResourceMemory" + + ", containerResourceMemory" + container.getResource().getMemory()) } } @@ -372,12 +372,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } /** - * Clean up the staging directory. + * Clean up the staging directory. */ - private def cleanupStagingDir() { + private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -393,7 +393,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { def run() { @@ -446,18 +446,18 @@ object ApplicationMaster { // Note that this will unfortunately not properly clean up the staging files because it gets // called too late, after the filesystem is already shutdown. if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } - } + } } ) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 29892e98e3..cc150888eb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -59,7 +59,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) // for client user who want to monitor app status by itself. def runApp() = { @@ -103,7 +103,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD) - ).foreach { case(cond, errStr) => + ).foreach { case(cond, errStr) => if (cond) { logError(errStr) args.printUsageAndExit(1) @@ -130,7 +130,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl queueInfo.getChildQueues.size)) } - def verifyClusterResources(app: GetNewApplicationResponse) = { + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) @@ -146,7 +146,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // We could add checks to make sure the entire cluster has enough resources but that involves - // getting all the node reports and computing ourselves + // getting all the node reports and computing ourselves } def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { @@ -207,7 +207,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) - } + } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) @@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -241,7 +241,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, + Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")) .foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -253,7 +253,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } } @@ -265,7 +265,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache, true) } } @@ -277,7 +277,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache) } } @@ -289,7 +289,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, linkname, statCache) } } @@ -299,7 +299,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def setupLaunchEnv( - localResources: HashMap[String, LocalResource], + localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) @@ -354,7 +354,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Add Xmx for am memory JAVA_OPTS += "-Xmx" + amMemory + "m " - JAVA_OPTS += " -Djava.io.tmpdir=" + + JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " // Commenting it out for now - so that people can refer to the properties if required. Remove @@ -387,11 +387,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } - val commands = List[String](javaCommand + + val commands = List[String](javaCommand + " -server " + JAVA_OPTS + " " + args.amClass + - " --class " + args.userClass + + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + " --worker-memory " + args.workerMemory + @@ -421,7 +421,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl super.submitApplication(appContext) } - def monitorApplication(appId: ApplicationId): Boolean = { + def monitorApplication(appId: ApplicationId): Boolean = { while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -443,7 +443,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() - if (state == YarnApplicationState.FINISHED || + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { return true @@ -461,7 +461,7 @@ object Client { def main(argStrings: Array[String]) { // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes - conf.set("SPARK_YARN_MODE", "true") + System.setProperty("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) @@ -479,25 +479,25 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + SPARK_JAR) Client.populateHadoopClasspath(conf, env) if (!userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + "*") } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 617289f568..e9e46a193b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,7 +33,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" -- cgit v1.2.3 From f4e40661912af2a23e250a49f72f00675172e2de Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 30 Dec 2013 11:13:24 -0800 Subject: Refactored kafka, flume, zeromq, mqtt as separate external projects, with their own self-contained scala API, java API, scala unit tests and java unit tests. Updated examples to use the external projects. --- .../streaming/examples/JavaFlumeEventCount.java | 11 +- .../streaming/examples/JavaKafkaWordCount.java | 4 +- .../spark/streaming/examples/FlumeEventCount.scala | 1 + .../spark/streaming/examples/KafkaWordCount.scala | 1 + .../spark/streaming/examples/MQTTWordCount.scala | 10 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 6 +- .../spark/streaming/flume/FlumeFunctions.scala | 43 ++++ .../spark/streaming/flume/FlumeInputDStream.scala | 155 +++++++++++++ .../flume/JavaStreamingContextWithFlume.scala | 48 ++++ .../org/apache/spark/streaming/flume/package.scala | 23 ++ .../flume/src/test/java/JavaFlumeStreamSuite.java | 38 ++++ external/flume/src/test/resources/log4j.properties | 29 +++ .../spark/streaming/flume/FlumeStreamSuite.scala | 86 ++++++++ .../kafka/JavaStreamingContextWithKafka.scala | 107 +++++++++ .../spark/streaming/kafka/KafkaFunctions.scala | 73 ++++++ .../spark/streaming/kafka/KafkaInputDStream.scala | 155 +++++++++++++ .../org/apache/spark/streaming/kafka/package.scala | 23 ++ .../streaming/kafka/JavaKafkaStreamSuite.java | 51 +++++ external/kafka/src/test/resources/log4j.properties | 29 +++ .../spark/streaming/kafka/KafkaStreamSuite.scala | 39 ++++ .../mqtt/JavaStreamingContextWithMQTT.scala | 59 +++++ .../spark/spark/streaming/mqtt/MQTTFunctions.scala | 43 ++++ .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 ++++++++++ .../spark/spark/streaming/mqtt/package.scala | 24 ++ .../spark/streaming/mqtt/JavaMQTTStreamSuite.java | 41 ++++ external/mqtt/src/test/resources/log4j.properties | 29 +++ .../spark/streaming/mqtt/MQTTStreamSuite.scala | 36 +++ .../twitter/JavaStreamingContextWithTwitter.scala | 99 +++++++++ .../twitter/StreamingContextWithTwitter.scala | 27 --- .../spark/streaming/twitter/TwitterFunctions.scala | 49 +++++ .../apache/spark/streaming/twitter/package.scala | 21 +- .../streaming/twitter/JavaTwitterStreamSuite.java | 51 +++++ .../twitter/src/test/resources/log4j.properties | 29 +++ .../streaming/twitter/TwitterStreamSuite.scala | 42 ++++ .../zeromq/JavaStreamingContextWithZeroMQ.scala | 102 +++++++++ .../spark/streaming/zeromq/ZeroMQFunctions.scala | 57 +++++ .../spark/streaming/zeromq/ZeroMQReceiver.scala | 54 +++++ .../apache/spark/streaming/zeromq/package.scala | 24 ++ .../streaming/zeromq/JavaZeroMQStreamSuite.java | 55 +++++ .../zeromq/src/test/resources/log4j.properties | 29 +++ .../spark/streaming/zeromq/ZeroMQStreamSuite.scala | 44 ++++ project/SparkBuild.scala | 89 +++++--- .../apache/spark/streaming/StreamingContext.scala | 129 +---------- .../streaming/api/java/JavaStreamingContext.scala | 244 +-------------------- .../streaming/dstream/FlumeInputDStream.scala | 154 ------------- .../streaming/dstream/KafkaInputDStream.scala | 153 ------------- .../spark/streaming/dstream/MQTTInputDStream.scala | 110 ---------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 53 ----- .../spark/streaming/scheduler/JobGenerator.scala | 1 + .../streaming/scheduler/NetworkInputTracker.scala | 5 +- .../org/apache/spark/streaming/JavaAPISuite.java | 80 +------ .../spark/streaming/LocalJavaStreamingContext.java | 46 ++++ .../apache/spark/streaming/InputStreamsSuite.scala | 75 +------ .../org/apache/spark/streaming/TestSuiteBase.scala | 9 +- 54 files changed, 2059 insertions(+), 1046 deletions(-) create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala create mode 100644 external/flume/src/test/java/JavaFlumeStreamSuite.java create mode 100644 external/flume/src/test/resources/log4j.properties create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala create mode 100644 external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java create mode 100644 external/kafka/src/test/resources/log4j.properties create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala create mode 100644 external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java create mode 100644 external/mqtt/src/test/resources/log4j.properties create mode 100644 external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala create mode 100644 external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java create mode 100644 external/twitter/src/test/resources/log4j.properties create mode 100644 external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala create mode 100644 external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java create mode 100644 external/zeromq/src/test/resources/log4j.properties create mode 100644 external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala create mode 100644 streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java (limited to 'examples') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f..e53c4f9e83 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -20,7 +20,8 @@ package org.apache.spark.streaming.examples; import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.dstream.SparkFlumeEvent; +import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; +import org.apache.spark.streaming.flume.SparkFlumeEvent; /** * Produces a count of events received from Flume. @@ -49,10 +50,10 @@ public class JavaFlumeEventCount { Duration batchInterval = new Duration(2000); - JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, + JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - - JavaDStream flumeStream = sc.flumeStream("localhost", port); + JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); + JavaDStream flumeStream = sscWithFlume.flumeStream("localhost", port); flumeStream.count(); @@ -63,6 +64,6 @@ public class JavaFlumeEventCount { } }).print(); - sc.start(); + ssc.start(); } } diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec..de0420ca83 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Duration; 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.kafka.JavaStreamingContextWithKafka; import scala.Tuple2; /** @@ -63,7 +64,8 @@ public class JavaKafkaWordCount { topicMap.put(topic, numThreads); } - JavaPairDStream messages = ssc.kafkaStream(args[1], args[2], topicMap); + JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); + JavaPairDStream messages = sscWithKafka.kafkaStream(args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 9f6e163454..f1641e299f 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples import org.apache.spark.util.IntParam import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ /** * Produces a count of events received from Flume. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 570ba4c81a..833c83a5ef 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -24,6 +24,7 @@ import kafka.producer._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.util.RawTextHelper._ +import org.apache.spark.streaming.kafka._ /** * Consumes messages from one or more topics in Kafka and does wordcount. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ff332a0282..edb46ac1da 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -17,11 +17,6 @@ package org.apache.spark.streaming.examples -import org.apache.spark.streaming.{ Seconds, StreamingContext } -import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.streaming.dstream.MQTTReceiver -import org.apache.spark.storage.StorageLevel - import org.eclipse.paho.client.mqttv3.MqttClient import org.eclipse.paho.client.mqttv3.MqttClientPersistence import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence @@ -29,6 +24,11 @@ import org.eclipse.paho.client.mqttv3.MqttException import org.eclipse.paho.client.mqttv3.MqttMessage import org.eclipse.paho.client.mqttv3.MqttTopic +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.mqtt._ + /** * A simple Mqtt publisher for demonstration purposes, repeatedly publishes * Space separated String Message "hello mqtt demo for spark streaming" 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 e83ce78aa5..037b96e9eb 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 @@ -20,11 +20,13 @@ package org.apache.spark.streaming.examples import akka.actor.ActorSystem import akka.actor.actorRef2Scala import akka.zeromq._ -import org.apache.spark.streaming.{ Seconds, StreamingContext } -import org.apache.spark.streaming.StreamingContext._ import akka.zeromq.Subscribe import akka.util.ByteString +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.zeromq._ + /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages * every one second. diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala new file mode 100644 index 0000000000..35e7a01abc --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala @@ -0,0 +1,43 @@ +/* + * 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.flume + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + +/** + * Extra Flume input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversion. Import org.apache.spark.streaming.flume._ to use these functions. + */ +class FlumeFunctions(ssc: StreamingContext) { + /** + * Create a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def flumeStream ( + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[SparkFlumeEvent] = { + val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala new file mode 100644 index 0000000000..ce3ef47cfe --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -0,0 +1,155 @@ +/* + * 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.flume + +import java.net.InetSocketAddress +import java.io.{ObjectInput, ObjectOutput, Externalizable} +import java.nio.ByteBuffer + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import org.apache.flume.source.avro.AvroSourceProtocol +import org.apache.flume.source.avro.AvroFlumeEvent +import org.apache.flume.source.avro.Status +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.avro.ipc.NettyServer + +import org.apache.spark.util.Utils +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ + +private[streaming] +class FlumeInputDStream[T: ClassTag]( + @transient ssc_ : StreamingContext, + host: String, + port: Int, + storageLevel: StorageLevel +) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { + + override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { + new FlumeReceiver(host, port, storageLevel) + } +} + +/** + * A wrapper class for AvroFlumeEvent's with a custom serialization format. + * + * This is necessary because AvroFlumeEvent uses inner data structures + * which are not serializable. + */ +class SparkFlumeEvent() extends Externalizable { + var event : AvroFlumeEvent = new AvroFlumeEvent() + + /* De-serialize from bytes. */ + def readExternal(in: ObjectInput) { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.read(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.read(keyBuff) + val key : String = Utils.deserialize(keyBuff) + + val valLength = in.readInt() + val valBuff = new Array[Byte](valLength) + in.read(valBuff) + val value : String = Utils.deserialize(valBuff) + + headers.put(key, value) + } + + event.setBody(ByteBuffer.wrap(bodyBuff)) + event.setHeaders(headers) + } + + /* Serialize to bytes. */ + def writeExternal(out: ObjectOutput) { + val body = event.getBody.array() + out.writeInt(body.length) + out.write(body) + + val numHeaders = event.getHeaders.size() + out.writeInt(numHeaders) + for ((k, v) <- event.getHeaders) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} + +private[streaming] object SparkFlumeEvent { + def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = { + val event = new SparkFlumeEvent + event.event = in + event + } +} + +/** A simple server that implements Flume's Avro protocol. */ +private[streaming] +class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { + override def append(event : AvroFlumeEvent) : Status = { + receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event) + Status.OK + } + + override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { + events.foreach (event => + receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)) + Status.OK + } +} + +/** A NetworkReceiver which listens for events using the + * Flume Avro interface.*/ +private[streaming] +class FlumeReceiver( + host: String, + port: Int, + storageLevel: StorageLevel + ) extends NetworkReceiver[SparkFlumeEvent] { + + lazy val blockGenerator = new BlockGenerator(storageLevel) + + protected override def onStart() { + val responder = new SpecificResponder( + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + val server = new NettyServer(responder, new InetSocketAddress(host, port)) + blockGenerator.start() + server.start() + logInfo("Flume receiver started") + } + + protected override def onStop() { + blockGenerator.stop() + logInfo("Flume receiver stopped") + } + + override def getLocationPreference = Some(host) +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala new file mode 100644 index 0000000000..4e66ae3535 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala @@ -0,0 +1,48 @@ +/* + * 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.flume + +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.storage.StorageLevel + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Flume input streams. + */ +class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext) + extends JavaStreamingContext(javaStreamingContext.ssc) { + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + */ + def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { + ssc.flumeStream(hostname, port) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): + JavaDStream[SparkFlumeEvent] = { + ssc.flumeStream(hostname, port, storageLevel) + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala new file mode 100644 index 0000000000..c087a39d1c --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala @@ -0,0 +1,23 @@ +/* + * 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 + +package object flume { + implicit def sscToFlumeFunctions(ssc: StreamingContext) = new FlumeFunctions(ssc) +} + diff --git a/external/flume/src/test/java/JavaFlumeStreamSuite.java b/external/flume/src/test/java/JavaFlumeStreamSuite.java new file mode 100644 index 0000000000..deffc78c4c --- /dev/null +++ b/external/flume/src/test/java/JavaFlumeStreamSuite.java @@ -0,0 +1,38 @@ +/* + * 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. + */ + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; +import org.apache.spark.streaming.flume.SparkFlumeEvent; +import org.junit.Test; + +public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); + + // tests the API, does not actually test data receiving + JavaDStream test1 = sscWithFlume.flumeStream("localhost", 12345); + JavaDStream test2 = sscWithFlume.flumeStream("localhost", 12345, + StorageLevel.MEMORY_AND_DISK_SER_2()); + + // To verify that JavaStreamingContextWithKafka is also StreamingContext + JavaDStream socketStream = sscWithFlume.socketTextStream("localhost", 9999); + } +} diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties new file mode 100644 index 0000000000..063529a9cb --- /dev/null +++ b/external/flume/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala new file mode 100644 index 0000000000..ba33320d02 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -0,0 +1,86 @@ +/* + * 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.flume + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} + +import java.net.InetSocketAddress +import java.nio.ByteBuffer +import java.nio.charset.Charset + +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.util.ManualClock + +class FlumeStreamSuite extends TestSuiteBase { + + val testPort = 9999 + + test("flume input stream") { + // Set up the streaming context and input streams + val ssc = new StreamingContext(master, framework, batchDuration) + val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + ssc.registerOutputStream(outputStream) + ssc.start() + + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5) + Thread.sleep(1000) + val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) + val client = SpecificRequestor.getClient( + classOf[AvroSourceProtocol], transceiver) + + for (i <- 0 until input.size) { + val event = new AvroFlumeEvent + event.setBody(ByteBuffer.wrap(input(i).toString.getBytes())) + event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) + client.append(event) + Thread.sleep(500) + clock.addToTime(batchDuration.milliseconds) + } + + val startTime = System.currentTimeMillis() + while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size) + Thread.sleep(100) + } + Thread.sleep(1000) + val timeTaken = System.currentTimeMillis() - startTime + assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") + logInfo("Stopping context") + ssc.stop() + + val decoder = Charset.forName("UTF-8").newDecoder() + + assert(outputBuffer.size === input.length) + for (i <- 0 until outputBuffer.size) { + assert(outputBuffer(i).size === 1) + val str = decoder.decode(outputBuffer(i).head.event.getBody) + assert(str.toString === input(i).toString) + assert(outputBuffer(i).head.event.getHeaders.get("test") === "header") + } + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala new file mode 100644 index 0000000000..ab0e8a6c8d --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala @@ -0,0 +1,107 @@ +/* + * 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.kafka + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import java.lang.{Integer => JInt} +import java.util.{Map => JMap} + +import kafka.serializer.Decoder + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Kafka input streams. + */ +class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) + extends JavaStreamingContext(javaStreamingContext.ssc) { + + /** + * Create an input stream that pulls messages form 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 + * in its own thread. + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages form 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 + * in its own thread. + * @param storageLevel RDD storage level. + * + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param keyTypeClass Key type of RDD + * @param valueTypeClass value type of RDD + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. Defaults to memory-only + */ + def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val valueCmt: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + + implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] + implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + + ssc.kafkaStream[K, V, U, T]( + kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala new file mode 100644 index 0000000000..2135634a69 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala @@ -0,0 +1,73 @@ +/* + * 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.kafka + +import scala.reflect.ClassTag + +import kafka.serializer.{Decoder, StringDecoder} + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + +/** + * Extra Kafka input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversion. Import org.apache.spark.streaming.kafka._ to use these functions. + */ +class KafkaFunctions(ssc: StreamingContext) { + /** + * Create an 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 + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + kafkaStream[String, String, StringDecoder, StringDecoder]( + kafkaParams, + topics, + storageLevel) + } + + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + */ + def kafkaStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): DStream[(K, V)] = { + val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala new file mode 100644 index 0000000000..fd69328aba --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -0,0 +1,155 @@ +/* + * 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.kafka + +import scala.collection.Map +import scala.reflect.ClassTag + +import java.util.Properties +import java.util.concurrent.Executors + +import kafka.consumer._ +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties +import kafka.utils.ZKStringSerializer +import org.I0Itec.zkclient._ + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ + + +/** + * Input stream that pulls messages from a Kafka Broker. + * + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ +private[streaming] +class KafkaInputDStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: Manifest, + T <: Decoder[_]: Manifest]( + @transient ssc_ : StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ) extends NetworkInputDStream[(K, V)](ssc_) with Logging { + + def getReceiver(): NetworkReceiver[(K, V)] = { + new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) + .asInstanceOf[NetworkReceiver[(K, V)]] + } +} + +private[streaming] +class KafkaReceiver[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: Manifest, + T <: Decoder[_]: Manifest]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ) extends NetworkReceiver[Any] { + + // Handles pushing data into the BlockManager + lazy protected val blockGenerator = new BlockGenerator(storageLevel) + // Connection to Kafka + var consumerConnector : ConsumerConnector = null + + def onStop() { + blockGenerator.stop() + } + + def onStart() { + + blockGenerator.start() + + // In case we are using multiple Threads to handle Kafka Messages + val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) + + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) + + // Kafka connection properties + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + + // Create the connection to the cluster + logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect")) + val consumerConfig = new ConsumerConfig(props) + consumerConnector = Consumer.create(consumerConfig) + logInfo("Connected to " + kafkaParams("zookeeper.connect")) + + // When autooffset.reset is defined, it is our responsibility to try and whack the + // consumer group zk node. + if (kafkaParams.contains("auto.offset.reset")) { + tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id")) + } + + val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[V]] + + // Create Threads for each Topic/Message Stream we are listening + val topicMessageStreams = consumerConnector.createMessageStreams( + topics, keyDecoder, valueDecoder) + + + // Start the messages handler for each partition + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } + } + } + + // Handles Kafka Messages + private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) + extends Runnable { + def run() { + logInfo("Starting MessageHandler.") + for (msgAndMetadata <- stream) { + blockGenerator += (msgAndMetadata.key, msgAndMetadata.message) + } + } + } + + // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because + // Kafka 0.7.2 only honors this param when the group is not in zookeeper. + // + // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' + // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest': + // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala + private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { + try { + val dir = "/consumers/" + groupId + logInfo("Cleaning up temporary zookeeper data under " + dir + ".") + val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + zk.deleteRecursive(dir) + zk.close() + } catch { + case _ : Throwable => // swallow + } + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala new file mode 100644 index 0000000000..44e7ce6e1b --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -0,0 +1,23 @@ +/* + * 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 + +package object kafka { + implicit def sscToKafkaFunctions(ssc: StreamingContext) = new KafkaFunctions(ssc) +} + diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java new file mode 100644 index 0000000000..66236df662 --- /dev/null +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -0,0 +1,51 @@ +/* + * 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.kafka; + +import java.util.HashMap; +import org.junit.Test; +import com.google.common.collect.Maps; +import kafka.serializer.StringDecoder; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; + +public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { + @Test + public void testKafkaStream() { + + HashMap topics = Maps.newHashMap(); + JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); + + // tests the API, does not actually test data receiving + JavaPairDStream test1 = sscWithKafka.kafkaStream("localhost:12345", "group", topics); + JavaPairDStream test2 = sscWithKafka.kafkaStream("localhost:12345", "group", topics, + StorageLevel.MEMORY_AND_DISK_SER_2()); + + HashMap kafkaParams = Maps.newHashMap(); + kafkaParams.put("zookeeper.connect","localhost:12345"); + kafkaParams.put("group.id","consumer-group"); + JavaPairDStream test3 = sscWithKafka.kafkaStream( + String.class, String.class, StringDecoder.class, StringDecoder.class, + kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); + + // To verify that JavaStreamingContextWithKafka is also StreamingContext + JavaDStream socketStream = sscWithKafka.socketTextStream("localhost", 9999); + } +} diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties new file mode 100644 index 0000000000..063529a9cb --- /dev/null +++ b/external/kafka/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala new file mode 100644 index 0000000000..2ef3e99c55 --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -0,0 +1,39 @@ +/* + * 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.kafka + +import kafka.serializer.StringDecoder +import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.storage.StorageLevel + +class KafkaStreamSuite extends TestSuiteBase { + + test("kafka input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topics = Map("my-topic" -> 1) + + // tests the API, does not actually test data receiving + val test1 = ssc.kafkaStream("localhost:12345", "group", topics) + val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") + val test3 = ssc.kafkaStream[String, String, StringDecoder, StringDecoder]( + kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) + + // TODO: Actually test receiving data + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala new file mode 100644 index 0000000000..d814da0f0d --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala @@ -0,0 +1,59 @@ +/* + * 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.mqtt + +import scala.reflect.ClassTag + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating MQTT input streams. + */ +class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext) + extends JavaStreamingContext(javaStreamingContext.ssc) { + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + */ + def mqttStream( + brokerUrl: String, + topic: String + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + ssc.mqttStream(brokerUrl, topic) + } + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. + */ + def mqttStream( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + ssc.mqttStream(brokerUrl, topic, storageLevel) + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala new file mode 100644 index 0000000000..86f4e9c724 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala @@ -0,0 +1,43 @@ +/* + * 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.mqtt + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + +/** + * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions. + */ +class MQTTFunctions(ssc: StreamingContext) { + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def mqttStream( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[String] = { + val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala new file mode 100644 index 0000000000..c8987a3ee0 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala @@ -0,0 +1,110 @@ +/* + * 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.mqtt + +import scala.collection.Map +import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import java.util.Properties +import java.util.concurrent.Executors +import java.io.IOException + +import org.eclipse.paho.client.mqttv3.MqttCallback +import org.eclipse.paho.client.mqttv3.MqttClient +import org.eclipse.paho.client.mqttv3.MqttClientPersistence +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken +import org.eclipse.paho.client.mqttv3.MqttException +import org.eclipse.paho.client.mqttv3.MqttMessage +import org.eclipse.paho.client.mqttv3.MqttTopic + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ + +/** + * Input stream that subscribe messages from a Mqtt Broker. + * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ + * @param brokerUrl Url of remote mqtt publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. + */ + +private[streaming] +class MQTTInputDStream[T: ClassTag]( + @transient ssc_ : StreamingContext, + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends NetworkInputDStream[T](ssc_) with Logging { + + def getReceiver(): NetworkReceiver[T] = { + new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] + } +} + +private[streaming] +class MQTTReceiver(brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends NetworkReceiver[Any] { + lazy protected val blockGenerator = new BlockGenerator(storageLevel) + + def onStop() { + blockGenerator.stop() + } + + def onStart() { + + blockGenerator.start() + + // Set up persistence for messages + var peristance: MqttClientPersistence = new MemoryPersistence() + + // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance + var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance) + + // Connect to MqttBroker + client.connect() + + // Subscribe to Mqtt topic + client.subscribe(topic) + + // Callback automatically triggers as and when new message arrives on specified topic + var callback: MqttCallback = new MqttCallback() { + + // Handles Mqtt message + override def messageArrived(arg0: String, arg1: MqttMessage) { + blockGenerator += new String(arg1.getPayload()) + } + + override def deliveryComplete(arg0: IMqttDeliveryToken) { + } + + override def connectionLost(arg0: Throwable) { + logInfo("Connection lost " + arg0) + } + } + + // Set up callback for MqttClient + client.setCallback(callback) + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala new file mode 100644 index 0000000000..28a944f57e --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala @@ -0,0 +1,24 @@ +/* + * 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 + +package object mqtt { + implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc) +} + + diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java new file mode 100644 index 0000000000..c1f41640dc --- /dev/null +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -0,0 +1,41 @@ +/* + * 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.mqtt; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.junit.Test; + +import org.apache.spark.streaming.LocalJavaStreamingContext; + +public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { + @Test + public void testMQTTStream() { + String brokerUrl = "abc"; + String topic = "def"; + JavaStreamingContextWithMQTT sscWithMQTT = new JavaStreamingContextWithMQTT(ssc); + + // tests the API, does not actually test data receiving + JavaDStream test1 = sscWithMQTT.mqttStream(brokerUrl, topic); + JavaDStream test2 = sscWithMQTT.mqttStream(brokerUrl, topic, + StorageLevel.MEMORY_AND_DISK_SER_2()); + + // To verify that JavaStreamingContextWithKafka is also StreamingContext + JavaDStream socketStream = sscWithMQTT.socketTextStream("localhost", 9999); + } +} diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties new file mode 100644 index 0000000000..063529a9cb --- /dev/null +++ b/external/mqtt/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala new file mode 100644 index 0000000000..ab6542918b --- /dev/null +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.mqtt + +import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.storage.StorageLevel + +class MQTTStreamSuite extends TestSuiteBase { + + test("MQTT input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val brokerUrl = "abc" + val topic = "def" + + // tests the API, does not actually test data receiving + val test1 = ssc.mqttStream(brokerUrl, topic) + val test2 = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + + // TODO: Actually test receiving data + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala new file mode 100644 index 0000000000..0250364331 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala @@ -0,0 +1,99 @@ +/* + * 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.twitter + +import twitter4j.Status +import twitter4j.auth.Authorization + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Twitter input streams. + */ +class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext) + extends JavaStreamingContext(javaStreamingContext.ssc) { + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + */ + def twitterStream(): JavaDStream[Status] = { + ssc.twitterStream(None) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream(filters: Array[String]): JavaDStream[Status] = { + ssc.twitterStream(None, filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { + ssc.twitterStream(None, filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization + */ + def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { + ssc.twitterStream(Some(twitterAuth)) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + twitterAuth: Authorization, + filters: Array[String] + ): JavaDStream[Status] = { + ssc.twitterStream(Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization object + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Authorization, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala deleted file mode 100644 index fe66e28ce6..0000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala +++ /dev/null @@ -1,27 +0,0 @@ -package org.apache.spark.streaming.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - - -class StreamingContextWithTwitter(ssc: StreamingContext) { - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth - * authorization; this uses the system properties twitter4j.oauth.consumerKey, - * .consumerSecret, .accessToken and .accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Option[Authorization] = None, - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { - val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala new file mode 100644 index 0000000000..e91049d9b1 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala @@ -0,0 +1,49 @@ +/* + * 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.twitter + +import twitter4j.Status +import twitter4j.auth.Authorization + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + +/** + * Extra Twitter input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversions. Import org.apache.spark.streaming.twitter._ to use these functions. + */ +class TwitterFunctions(ssc: StreamingContext) { + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Option[Authorization], + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = { + val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala index 89c202a730..23f82c5885 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala @@ -1,7 +1,22 @@ +/* + * 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 package object twitter { - implicit def enrichMyStreamingContext(ssc: StreamingContext): StreamingContextWithTwitter = { - new StreamingContextWithTwitter(ssc) - } + implicit def sscToTwitterFunctions(ssc: StreamingContext) = new TwitterFunctions(ssc) } diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java new file mode 100644 index 0000000000..34e4fbdd85 --- /dev/null +++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java @@ -0,0 +1,51 @@ +/* + * 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.twitter; + +import java.util.Arrays; +import org.junit.Test; + +import twitter4j.Status; +import twitter4j.auth.Authorization; +import twitter4j.auth.NullAuthorization; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; + +public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { + @Test + public void testTwitterStream() { + JavaStreamingContextWithTwitter sscWithTwitter = new JavaStreamingContextWithTwitter(ssc); + String[] filters = (String[])Arrays.asList("filter1", "filter2").toArray(); + Authorization auth = NullAuthorization.getInstance(); + + // tests the API, does not actually test data receiving + JavaDStream test1 = sscWithTwitter.twitterStream(); + JavaDStream test2 = sscWithTwitter.twitterStream(filters); + JavaDStream test3 = + sscWithTwitter.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test4 = sscWithTwitter.twitterStream(auth); + JavaDStream test5 = sscWithTwitter.twitterStream(auth, filters); + JavaDStream test6 = + sscWithTwitter.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + + // To verify that JavaStreamingContextWithKafka is also StreamingContext + JavaDStream socketStream = sscWithTwitter.socketTextStream("localhost", 9999); + } +} diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties new file mode 100644 index 0000000000..063529a9cb --- /dev/null +++ b/external/twitter/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala new file mode 100644 index 0000000000..d7f6d35e07 --- /dev/null +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -0,0 +1,42 @@ +/* + * 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.twitter + +import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.storage.StorageLevel +import twitter4j.auth.{NullAuthorization, Authorization} + +class TwitterStreamSuite extends TestSuiteBase { + + test("kafka input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val filters = Seq("filter1", "filter2") + val authorization: Authorization = NullAuthorization.getInstance() + + // tests the API, does not actually test data receiving + val test1 = ssc.twitterStream(None) + val test2 = ssc.twitterStream(None, filters) + val test3 = ssc.twitterStream(None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4 = ssc.twitterStream(Some(authorization)) + val test5 = ssc.twitterStream(Some(authorization), filters) + val test6 = ssc.twitterStream(Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) + + // Note that actually testing the data receiving is hard as authentication keys are + // necessary for accessing Twitter live stream + } +} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala new file mode 100644 index 0000000000..dc5d1f05be --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala @@ -0,0 +1,102 @@ +/* + * 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.zeromq + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import akka.actor.SupervisorStrategy +import akka.util.ByteString +import akka.zeromq.Subscribe + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating ZeroMQ input streams. + */ +class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) + extends JavaStreamingContext(javaStreamingContext.ssc) { + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote ZeroMQ publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel Storage level to use for storing the received objects + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel, + supervisorStrategy: SupervisorStrategy + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + ssc.zeroMQStream[T](publisherUrl, subscribe, fn) + } +} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala new file mode 100644 index 0000000000..f4c75ab7c9 --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala @@ -0,0 +1,57 @@ +/* + * 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.zeromq + +import scala.reflect.ClassTag + +import akka.actor.{Props, SupervisorStrategy} +import akka.util.ByteString +import akka.zeromq.Subscribe + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.receivers._ + +/** + * Extra ZeroMQ input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversions. Import org.apache.spark.streaming.zeromq._ to use these functions. + */ +class ZeroMQFunctions(ssc: StreamingContext) { + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic + * and each frame has sequence of byte thus it needs the converter + * (which might be deserializer of bytes) to translate from sequence + * of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. Defaults to memory-only. + */ + def zeroMQStream[T: ClassTag]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] ⇒ Iterator[T], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { + ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), + "ZeroMQReceiver", storageLevel, supervisorStrategy) + } +} + diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala new file mode 100644 index 0000000000..769761e3b8 --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -0,0 +1,54 @@ +/* + * 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.zeromq + +import scala.reflect.ClassTag + +import akka.actor.Actor +import akka.util.ByteString +import akka.zeromq._ + +import org.apache.spark.Logging +import org.apache.spark.streaming.receivers._ + +/** + * A receiver to subscribe to ZeroMQ stream. + */ +private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] ⇒ Iterator[T]) + extends Actor with Receiver with Logging { + + override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), + Connect(publisherUrl), subscribe) + + def receive: Receive = { + + case Connecting ⇒ logInfo("connecting ...") + + case m: ZMQMessage ⇒ + logDebug("Received message for:" + m.frame(0)) + + //We ignore first frame for processing as it is the topic + val bytes = m.frames.tail + pushBlock(bytesToObjects(bytes)) + + case Closed ⇒ logInfo("received closed ") + + } +} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala new file mode 100644 index 0000000000..dc27178149 --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala @@ -0,0 +1,24 @@ +/* + * 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 + +package object zeromq { + implicit def sscToZeroMQFunctions(ssc: StreamingContext) = new ZeroMQFunctions(ssc) +} + + diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java new file mode 100644 index 0000000000..96af7d737d --- /dev/null +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -0,0 +1,55 @@ +/* + * 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.zeromq; + +import org.junit.Test; + +import akka.actor.SupervisorStrategy; +import akka.util.ByteString; +import akka.zeromq.Subscribe; + +import org.apache.spark.api.java.function.Function; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; + +public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { + + @Test // tests the API, does not actually test data receiving + public void testZeroMQStream() { + JavaStreamingContextWithZeroMQ sscWithZeroMQ = new JavaStreamingContextWithZeroMQ(ssc); + String publishUrl = "abc"; + Subscribe subscribe = new Subscribe((ByteString)null); + Function> bytesToObjects = new Function>() { + @Override + public Iterable call(byte[][] bytes) throws Exception { + return null; + } + }; + + JavaDStream test1 = sscWithZeroMQ.zeroMQStream( + publishUrl, subscribe, bytesToObjects); + JavaDStream test2 = sscWithZeroMQ.zeroMQStream( + publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test3 = sscWithZeroMQ.zeroMQStream( + publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); + + // To verify that JavaStreamingContextWithKafka is also StreamingContext + JavaDStream socketStream = sscWithZeroMQ.socketTextStream("localhost", 9999); + } +} diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties new file mode 100644 index 0000000000..063529a9cb --- /dev/null +++ b/external/zeromq/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala new file mode 100644 index 0000000000..5adcdb821f --- /dev/null +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -0,0 +1,44 @@ +/* + * 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.zeromq + +import akka.actor.SupervisorStrategy +import akka.util.ByteString +import akka.zeromq.Subscribe + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} + +class ZeroMQStreamSuite extends TestSuiteBase { + + test("zeromq input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val publishUrl = "abc" + val subscribe = new Subscribe(null.asInstanceOf[ByteString]) + val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] + + // tests the API, does not actually test data receiving + val test1 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects) + val test2 = ssc.zeroMQStream( + publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) + val test3 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects, + StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) + + // TODO: Actually test data receiving + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f9ff781f38..4e92d2a162 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -49,9 +49,6 @@ object SparkBuild extends Build { lazy val repl = Project("repl", file("repl"), settings = replSettings) .dependsOn(core, bagel, mllib) - lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, bagel, streaming, externalTwitter) - lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) @@ -60,8 +57,6 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) - lazy val externalTwitter = Project("streaming-twitter", file("external/twitter"), settings = twitterSettings) dependsOn(streaming) - lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) @@ -94,10 +89,31 @@ object SparkBuild extends Build { lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) + .dependsOn(streaming % "compile->compile;test->test") + + lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings) + .dependsOn(streaming % "compile->compile;test->test") + + lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings) + .dependsOn(streaming % "compile->compile;test->test") + + lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings) + .dependsOn(streaming % "compile->compile;test->test") + + lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings) + .dependsOn(streaming % "compile->compile;test->test") + + lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) + lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) + + lazy val examples = Project("examples", file("examples"), settings = examplesSettings) + .dependsOn(core, mllib, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) + // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef - lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj) + lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", @@ -167,7 +183,7 @@ object SparkBuild extends Build { ), -/* + /* publishTo <<= version { (v: String) => val nexus = "https://oss.sonatype.org/" if (v.trim.endsWith("SNAPSHOT")) @@ -176,8 +192,7 @@ object SparkBuild extends Build { Some("sonatype-staging" at nexus + "service/local/staging/deploy/maven2") }, -*/ - + */ libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.0.CR1", @@ -264,7 +279,6 @@ object SparkBuild extends Build { libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) ) - def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", libraryDependencies ++= Seq( @@ -302,21 +316,10 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( - "Eclipse Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", "Apache repo" at "https://repository.apache.org/content/repositories/releases" ), - libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), - "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" - exclude("com.sun.jdmk", "jmxtools") - exclude("com.sun.jmx", "jmxri") - exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty), - "org.eclipse.paho" % "mqtt-client" % "0.4.0", - "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), - // "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) + "commons-io" % "commons-io" % "2.4" ) ) @@ -331,8 +334,8 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib) ) @@ -358,9 +361,45 @@ object SparkBuild extends Build { ) def twitterSettings() = streamingSettings ++ Seq( - name := "spark-twitter", + name := "spark-streaming-twitter", libraryDependencies ++= Seq( "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) ) ) + + def kafkaSettings() = streamingSettings ++ Seq( + name := "spark-streaming-kafka", + libraryDependencies ++= Seq( + "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), + "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" + exclude("com.sun.jdmk", "jmxtools") + exclude("com.sun.jmx", "jmxri") + exclude("net.sf.jopt-simple", "jopt-simple") + excludeAll(excludeNetty) + ) + ) + + def flumeSettings() = streamingSettings ++ Seq( + name := "spark-streaming-flume", + libraryDependencies ++= Seq( + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy) + ) + ) + + def zeromqSettings() = streamingSettings ++ Seq( + name := "spark-streaming-zeromq", + libraryDependencies ++= Seq( + "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) + ) + ) + + def mqttSettings() = streamingSettings ++ Seq( + name := "spark-streaming-mqtt", + resolvers ++= Seq( + "Apache repo" at "https://repository.apache.org/content/repositories/releases" + ), + libraryDependencies ++= Seq( + "org.eclipse.paho" % "mqtt-client" % "0.4.0" + ) + ) } 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 25b9b70b2c..41898b9228 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -17,21 +17,6 @@ package org.apache.spark.streaming -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.zeromq.Subscribe - -import org.apache.spark.streaming.dstream._ - -import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.receivers.ActorReceiver -import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy -import org.apache.spark.streaming.receivers.ZeroMQReceiver -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner -import org.apache.spark.streaming.receivers.ActorReceiver - import scala.collection.mutable.Queue import scala.collection.Map import scala.reflect.ClassTag @@ -40,15 +25,22 @@ import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger import java.util.UUID +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.MetadataCleaner +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.scheduler._ + import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path -//import twitter4j.Status -//import twitter4j.auth.Authorization -import org.apache.spark.streaming.scheduler._ -import akka.util.ByteString + +import akka.actor.Props +import akka.actor.SupervisorStrategy /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -223,74 +215,6 @@ class StreamingContext private ( networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic - * and each frame has sequence of byte thus it needs the converter - * (which might be deserializer of bytes) to translate from sequence - * of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to memory-only. - */ - def zeroMQStream[T: ClassTag]( - publisherUrl:String, - subscribe: Subscribe, - 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)), - "ZeroMQReceiver", storageLevel, supervisorStrategy) - } - - /** - * Create an 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 - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 - ): DStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - kafkaStream[String, String, kafka.serializer.StringDecoder, kafka.serializer.StringDecoder]( - kafkaParams, - topics, - storageLevel) - } - - /** - * Create an input stream that pulls messages from a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def kafkaStream[ - K: ClassTag, - V: ClassTag, - U <: kafka.serializer.Decoder[_]: Manifest, - T <: kafka.serializer.Decoder[_]: Manifest]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): DStream[(K, V)] = { - val inputStream = new KafkaInputDStream[K, V, U, T](this, kafkaParams, topics, storageLevel) - registerInputStream(inputStream) - inputStream - } - /** * Create a input stream from TCP source hostname:port. Data is received using * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited @@ -329,22 +253,6 @@ class StreamingContext private ( inputStream } - /** - * Create a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream ( - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream[SparkFlumeEvent](this, hostname, port, storageLevel) - registerInputStream(inputStream) - inputStream - } - /** * Create a input stream from network source hostname:port, where data is received * as serialized blocks (serialized using the Spark's serializer) that can be directly @@ -467,21 +375,6 @@ class StreamingContext private ( inputStream } -/** - * Create an input stream that receives messages pushed by a mqtt publisher. - * @param brokerUrl Url of remote mqtt publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to memory-only. - */ - - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[String] = { - val inputStream = new MQTTInputDStream[String](this, brokerUrl, topic, storageLevel) - registerInputStream(inputStream) - inputStream - } /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ 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 b32cfbb677..ea4a0fe619 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 @@ -17,28 +17,21 @@ package org.apache.spark.streaming.api.java -import java.lang.{Integer => JInt} -import java.io.InputStream -import java.util.{Map => JMap, List => JList} - import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import java.io.InputStream +import java.util.{Map => JMap, List => JList} + import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -//import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy -import akka.zeromq.Subscribe -import akka.util.ByteString - -//import twitter4j.auth.Authorization import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.scheduler.StreamingListener /** @@ -133,81 +126,6 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** The underlying SparkContext */ val sc: JavaSparkContext = new JavaSparkContext(ssc.sc) - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt]) - : JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), - StorageLevel.MEMORY_ONLY_SER_2) - - } - - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - * @param storageLevel RDD storage level. Defaults to memory-only - * - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel) - : JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. Defaults to memory-only - */ - def kafkaStream[K, V, U <: kafka.serializer.Decoder[_], T <: kafka.serializer.Decoder[_]]( - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel) - : JavaPairDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] - - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] - - ssc.kafkaStream[K, V, U, T]( - kafkaParams.toMap, - Map(topics.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - /** * Create a input stream from network source hostname:port. Data is received using * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited @@ -318,98 +236,6 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.fileStream[K, V, F](directory) } - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): - JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port, storageLevel) - } - - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port) - } - /* - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String], - storageLevel: StorageLevel - ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * .consumerSecret, .accessToken and .accessTokenSecret to be set. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - filters: Array[String], - storageLevel: StorageLevel - ): JavaDStream[Status] = { - ssc.twitterStream(None, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String] - ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * .consumerSecret, .accessToken and .accessTokenSecret to be set. - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream( - filters: Array[String] - ): JavaDStream[Status] = { - ssc.twitterStream(None, filters) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - */ - def twitterStream( - twitterAuth: Authorization - ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth)) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * .consumerSecret, .accessToken and .accessTokenSecret to be set. - */ - def twitterStream(): JavaDStream[Status] = { - ssc.twitterStream() - } - */ /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor @@ -472,70 +298,6 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.actorStream[T](props, name) } - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel Storage level to use for storing the received objects - */ - def zeroMQStream[T]( - publisherUrl:String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] ⇒ Iterator[T], - storageLevel: StorageLevel, - supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to memory-only. - */ - def zeroMQStream[T]( - publisherUrl:String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - */ - def zeroMQStream[T]( - publisherUrl:String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn) - } - /** * Registers an output stream that will be computed every interval */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala deleted file mode 100644 index 60d79175f1..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala +++ /dev/null @@ -1,154 +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 java.net.InetSocketAddress -import java.io.{ObjectInput, ObjectOutput, Externalizable} -import java.nio.ByteBuffer - -import scala.collection.JavaConversions._ -import scala.reflect.ClassTag - -import org.apache.flume.source.avro.AvroSourceProtocol -import org.apache.flume.source.avro.AvroFlumeEvent -import org.apache.flume.source.avro.Status -import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.avro.ipc.NettyServer - -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.util.Utils -import org.apache.spark.storage.StorageLevel - -private[streaming] -class FlumeInputDStream[T: ClassTag]( - @transient ssc_ : StreamingContext, - host: String, - port: Int, - storageLevel: StorageLevel -) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { - - override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { - new FlumeReceiver(host, port, storageLevel) - } -} - -/** - * A wrapper class for AvroFlumeEvent's with a custom serialization format. - * - * This is necessary because AvroFlumeEvent uses inner data structures - * which are not serializable. - */ -class SparkFlumeEvent() extends Externalizable { - var event : AvroFlumeEvent = new AvroFlumeEvent() - - /* De-serialize from bytes. */ - def readExternal(in: ObjectInput) { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.read(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.read(keyBuff) - val key : String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.read(valBuff) - val value : String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - - event.setBody(ByteBuffer.wrap(bodyBuff)) - event.setHeaders(headers) - } - - /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput) { - val body = event.getBody.array() - out.writeInt(body.length) - out.write(body) - - val numHeaders = event.getHeaders.size() - out.writeInt(numHeaders) - for ((k, v) <- event.getHeaders) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } - } -} - -private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = { - val event = new SparkFlumeEvent - event.event = in - event - } -} - -/** A simple server that implements Flume's Avro protocol. */ -private[streaming] -class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { - override def append(event : AvroFlumeEvent) : Status = { - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event) - Status.OK - } - - override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { - events.foreach (event => - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)) - Status.OK - } -} - -/** A NetworkReceiver which listens for events using the - * Flume Avro interface.*/ -private[streaming] -class FlumeReceiver( - host: String, - port: Int, - storageLevel: StorageLevel - ) extends NetworkReceiver[SparkFlumeEvent] { - - lazy val blockGenerator = new BlockGenerator(storageLevel) - - protected override def onStart() { - val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - val server = new NettyServer(responder, new InetSocketAddress(host, port)) - blockGenerator.start() - server.start() - logInfo("Flume receiver started") - } - - protected override def onStop() { - blockGenerator.stop() - logInfo("Flume receiver stopped") - } - - override def getLocationPreference = Some(host) -} 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 deleted file mode 100644 index 526f5564c7..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala +++ /dev/null @@ -1,153 +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.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext - -import java.util.Properties -import java.util.concurrent.Executors - -import kafka.consumer._ -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient._ - -import scala.collection.Map -import scala.reflect.ClassTag - -/** - * Input stream that pulls messages from a Kafka Broker. - * - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ -private[streaming] -class KafkaInputDStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( - @transient ssc_ : StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ) extends NetworkInputDStream[(K, V)](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[(K, V)] = { - new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[NetworkReceiver[(K, V)]] - } -} - -private[streaming] -class KafkaReceiver[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - - // Handles pushing data into the BlockManager - lazy protected val blockGenerator = new BlockGenerator(storageLevel) - // Connection to Kafka - var consumerConnector : ConsumerConnector = null - - def onStop() { - blockGenerator.stop() - } - - def onStart() { - - blockGenerator.start() - - // In case we are using multiple Threads to handle Kafka Messages - val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) - - logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) - - // Kafka connection properties - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - - // Create the connection to the cluster - logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect")) - val consumerConfig = new ConsumerConfig(props) - consumerConnector = Consumer.create(consumerConfig) - logInfo("Connected to " + kafkaParams("zookeeper.connect")) - - // When autooffset.reset is defined, it is our responsibility to try and whack the - // consumer group zk node. - if (kafkaParams.contains("auto.offset.reset")) { - tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id")) - } - - val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[V]] - - // Create Threads for each Topic/Message Stream we are listening - val topicMessageStreams = consumerConnector.createMessageStreams( - topics, keyDecoder, valueDecoder) - - - // Start the messages handler for each partition - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } - } - } - - // Handles Kafka Messages - private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) - extends Runnable { - def run() { - logInfo("Starting MessageHandler.") - for (msgAndMetadata <- stream) { - blockGenerator += (msgAndMetadata.key, msgAndMetadata.message) - } - } - } - - // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because - // Kafka 0.7.2 only honors this param when the group is not in zookeeper. - // - // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' - // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest': - // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala - private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { - try { - val dir = "/consumers/" + groupId - logInfo("Cleaning up temporary zookeeper data under " + dir + ".") - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) - zk.deleteRecursive(dir) - zk.close() - } catch { - case _ : Throwable => // swallow - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala deleted file mode 100644 index ef4a737568..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala +++ /dev/null @@ -1,110 +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.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{ Time, DStreamCheckpointData, StreamingContext } - -import java.util.Properties -import java.util.concurrent.Executors -import java.io.IOException - -import org.eclipse.paho.client.mqttv3.MqttCallback -import org.eclipse.paho.client.mqttv3.MqttClient -import org.eclipse.paho.client.mqttv3.MqttClientPersistence -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken -import org.eclipse.paho.client.mqttv3.MqttException -import org.eclipse.paho.client.mqttv3.MqttMessage -import org.eclipse.paho.client.mqttv3.MqttTopic - -import scala.collection.Map -import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ -import scala.reflect.ClassTag - -/** - * Input stream that subscribe messages from a Mqtt Broker. - * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ - * @param brokerUrl Url of remote mqtt publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - -private[streaming] -class MQTTInputDStream[T: ClassTag]( - @transient ssc_ : StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[T] = { - new MQTTReceiver(brokerUrl, topic, storageLevel) - .asInstanceOf[NetworkReceiver[T]] - } -} - -private[streaming] -class MQTTReceiver(brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) - - def onStop() { - blockGenerator.stop() - } - - def onStart() { - - blockGenerator.start() - - // Set up persistence for messages - var peristance: MqttClientPersistence = new MemoryPersistence() - - // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance) - - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - - // Callback automatically triggers as and when new message arrives on specified topic - var callback: MqttCallback = new MqttCallback() { - - // Handles Mqtt message - override def messageArrived(arg0: String, arg1: MqttMessage) { - blockGenerator += new String(arg1.getPayload()) - } - - override def deliveryComplete(arg0: IMqttDeliveryToken) { - } - - override def connectionLost(arg0: Throwable) { - logInfo("Connection lost " + arg0) - } - } - - // Set up callback for MqttClient - client.setCallback(callback) - } -} 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 deleted file mode 100644 index f164d516b0..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala +++ /dev/null @@ -1,53 +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.receivers - -import scala.reflect.ClassTag - -import akka.actor.Actor -import akka.util.ByteString -import akka.zeromq._ - -import org.apache.spark.Logging - -/** - * A receiver to subscribe to ZeroMQ stream. - */ -private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] ⇒ Iterator[T]) - extends Actor with Receiver with Logging { - - override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), - Connect(publisherUrl), subscribe) - - def receive: Receive = { - - case Connecting ⇒ logInfo("connecting ...") - - case m: ZMQMessage ⇒ - logDebug("Received message for:" + m.frame(0)) - - //We ignore first frame for processing as it is the topic - val bytes = m.frames.tail - pushBlock(bytesToObjects(bytes)) - - case Closed ⇒ logInfo("received closed ") - - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 1cd0b9b0a4..2734393ae9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -33,6 +33,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val ssc = jobScheduler.ssc val clockClass = System.getProperty( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") + logInfo("Using clock class = " + clockClass) val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, longTime => generateJobs(new Time(longTime))) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index abff55d77c..4a8e15db21 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -160,7 +160,10 @@ class NetworkInputTracker( } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. - ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + if (!ssc.sparkContext.isLocal) { + ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + } + // Distribute the receivers and start them ssc.sparkContext.runJob(tempRDD, startReceiver) 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 daeb99f5b7..f4d26c0be6 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,22 +17,16 @@ package org.apache.spark.streaming; -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 scala.Tuple2; -import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; -import org.apache.spark.streaming.api.java.JavaDStreamLike; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import java.io.*; +import java.util.*; -import scala.Tuple2; -import twitter4j.Status; +import com.google.common.base.Optional; +import com.google.common.collect.Lists; +import com.google.common.io.Files; import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.JavaPairRDD; @@ -43,39 +37,11 @@ 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 java.io.*; -import java.util.*; - -import akka.actor.Props; -import akka.zeromq.Subscribe; - // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. -public class JavaAPISuite implements Serializable { - private transient JavaStreamingContext ssc; - - @Before - public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port"); - } - +public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { @Test public void testCount() { List> inputData = Arrays.asList( @@ -1596,26 +1562,6 @@ public class JavaAPISuite implements Serializable { // Input stream tests. These mostly just test that we can instantiate a given InputStream with // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the // InputStream functionality is deferred to the existing Scala tests. - @Test - public void testKafkaStream() { - HashMap topics = Maps.newHashMap(); - JavaPairDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaPairDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, - StorageLevel.MEMORY_AND_DISK()); - - HashMap kafkaParams = Maps.newHashMap(); - kafkaParams.put("zookeeper.connect","localhost:12345"); - kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = ssc.kafkaStream( - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - topics, - StorageLevel.MEMORY_AND_DISK()); - } - @Test public void testSocketTextStream() { JavaDStream test = ssc.socketTextStream("localhost", 12345); @@ -1654,16 +1600,10 @@ public class JavaAPISuite implements Serializable { public void testRawSocketStream() { JavaDStream test = ssc.rawSocketStream("localhost", 12345); } - - @Test - public void testFlumeStream() { - JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); - } - + /* @Test public void testFileStream() { - JavaPairDStream foo = - ssc.>fileStream("/tmp/foo"); + JavaPairDStream foo = ssc.>fileStream("/tmp/foo"); } @Test @@ -1685,5 +1625,5 @@ public class JavaAPISuite implements Serializable { return null; } }); - } + } */ } diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java new file mode 100644 index 0000000000..34bee56885 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -0,0 +1,46 @@ +/* + * 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; + +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.junit.After; +import org.junit.Before; + +public abstract class LocalJavaStreamingContext { + + protected transient JavaStreamingContext ssc; + + @Before + public void setUp() { + System.clearProperty("spark.driver.port"); + System.clearProperty("spark.hostPort"); + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + ssc.checkpoint("checkpoint"); + } + + @After + public void tearDown() { + ssc.stop(); + ssc = null; + + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port"); + System.clearProperty("spark.hostPort"); + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 62a9f120b4..0cffed64a7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -23,7 +23,7 @@ import akka.actor.IOManager import akka.actor.Props import akka.util.ByteString -import org.apache.spark.streaming.dstream.{NetworkReceiver, SparkFlumeEvent} +import org.apache.spark.streaming.dstream.{NetworkReceiver} import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket} import java.io.{File, BufferedWriter, OutputStreamWriter} import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} @@ -31,18 +31,11 @@ import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.ManualClock import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.receivers.Receiver -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.Logging import scala.util.Random import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter -import org.apache.flume.source.avro.AvroSourceProtocol -import org.apache.flume.source.avro.AvroFlumeEvent -import org.apache.flume.source.avro.Status -import org.apache.avro.ipc.{specific, NettyTransceiver} -import org.apache.avro.ipc.specific.SpecificRequestor -import java.nio.ByteBuffer import collection.JavaConversions._ -import java.nio.charset.Charset import com.google.common.io.Files import java.util.concurrent.atomic.AtomicInteger @@ -99,55 +92,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } - test("flume input stream") { - // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) - val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) - ssc.registerOutputStream(outputStream) - ssc.start() - - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = Seq(1, 2, 3, 4, 5) - Thread.sleep(1000) - val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) - val client = SpecificRequestor.getClient( - classOf[AvroSourceProtocol], transceiver) - - for (i <- 0 until input.size) { - val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(input(i).toString.getBytes())) - event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) - client.append(event) - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - } - - val startTime = System.currentTimeMillis() - while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { - logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size) - Thread.sleep(100) - } - Thread.sleep(1000) - val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") - logInfo("Stopping context") - ssc.stop() - - val decoder = Charset.forName("UTF-8").newDecoder() - - assert(outputBuffer.size === input.length) - for (i <- 0 until outputBuffer.size) { - assert(outputBuffer(i).size === 1) - val str = decoder.decode(outputBuffer(i).head.event.getBody) - assert(str.toString === input(i).toString) - assert(outputBuffer(i).head.event.getHeaders.get("test") === "header") - } - } - - test("file input stream") { // Disable manual clock as FileInputDStream does not work with manual clock System.clearProperty("spark.streaming.clock") @@ -249,21 +193,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val topics = Map("my-topic" -> 1) - val test1 = ssc.kafkaStream("localhost:12345", "group", topics) - val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) - - // Test specifying decoder - val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = ssc.kafkaStream[ - String, - String, - kafka.serializer.StringDecoder, - kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK) - } - test("multi-thread receiver") { // set up the test receiver val numThreads = 10 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index e969e91d13..f56c0462f4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -137,11 +137,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // if you want to add your stuff to "before" (i.e., don't call before { } ) def beforeFunction() { if (useManualClock) { - System.setProperty( - "spark.streaming.clock", - "org.apache.spark.streaming.util.ManualClock" - ) + logInfo("Using manual clock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } else { + logInfo("Using real clock") System.clearProperty("spark.streaming.clock") } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown @@ -273,7 +272,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { val startTime = System.currentTimeMillis() while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput) - Thread.sleep(100) + Thread.sleep(10) } val timeTaken = System.currentTimeMillis() - startTime -- cgit v1.2.3 From 97630849ff2cdaa2ff8a115c3e8e6ca8dba7477d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 31 Dec 2013 00:28:57 -0800 Subject: Added pom.xml for external projects and removed unnecessary dependencies and repositoris from other poms and sbt. --- examples/pom.xml | 45 +++++++++++++------- external/flume/pom.xml | 93 ++++++++++++++++++++++++++++++++++++++++ external/kafka/pom.xml | 97 ++++++++++++++++++++++++++++++++++++++++++ external/mqtt/pom.xml | 108 +++++++++++++++++++++++++++++++++++++++++++++++ external/twitter/pom.xml | 89 ++++++++++++++++++++++++++++++++++++++ external/zeromq/pom.xml | 89 ++++++++++++++++++++++++++++++++++++++ pom.xml | 29 +++---------- project/SparkBuild.scala | 21 +++------ streaming/pom.xml | 83 +++++++++++++----------------------- 9 files changed, 548 insertions(+), 106 deletions(-) create mode 100644 external/flume/pom.xml create mode 100644 external/kafka/pom.xml create mode 100644 external/mqtt/pom.xml create mode 100644 external/twitter/pom.xml create mode 100644 external/zeromq/pom.xml (limited to 'examples') diff --git a/examples/pom.xml b/examples/pom.xml index 7a7032c319..1839667fea 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -71,6 +71,36 @@ ${project.version} provided + + org.apache.spark + spark-streaming-twitter_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming-kafka_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming-flume_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming-zeromq_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming-mqtt_${scala.binary.version} + ${project.version} + provided + org.apache.hbase hbase @@ -86,21 +116,6 @@ - - com.sksamuel.kafka - kafka_${scala.binary.version} - 0.8.0-beta1 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - org.eclipse.jetty jetty-server diff --git a/external/flume/pom.xml b/external/flume/pom.xml new file mode 100644 index 0000000000..443910a03a --- /dev/null +++ b/external/flume/pom.xml @@ -0,0 +1,93 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-flume_2.10 + jar + Spark Project External Flume + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.flume + flume-ng-sdk + 1.2.0 + + + org.jboss.netty + netty + + + org.xerial.snappy + * + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml new file mode 100644 index 0000000000..f782e0e126 --- /dev/null +++ b/external/kafka/pom.xml @@ -0,0 +1,97 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka_2.10 + jar + Spark Project External Kafka + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + com.sksamuel.kafka + kafka_${scala.binary.version} + 0.8.0-beta1 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml new file mode 100644 index 0000000000..31b4fa87de --- /dev/null +++ b/external/mqtt/pom.xml @@ -0,0 +1,108 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-mqtt_2.10 + jar + Spark Project External MQTT + http://spark.incubator.apache.org/ + + + + mqtt-repo + MQTT Repository + https://repo.eclipse.org/content/repositories/paho-releases + + true + + + false + + + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + org.eclipse.paho + mqtt-client + 0.4.0 + + + ${akka.group} + akka-zeromq_${scala.binary.version} + ${akka.version} + + + org.jboss.netty + netty + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml new file mode 100644 index 0000000000..216e6c1d8f --- /dev/null +++ b/external/twitter/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-twitter_2.10 + jar + Spark Project External Twitter + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + org.twitter4j + twitter4j-stream + 3.0.3 + + + org.jboss.netty + netty + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml new file mode 100644 index 0000000000..c240d59574 --- /dev/null +++ b/external/zeromq/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-zeromq_2.10 + jar + Spark Project External ZeroMQ + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + ${akka.group} + akka-zeromq_${scala.binary.version} + ${akka.version} + + + org.jboss.netty + netty + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + com.novocode + junit-interface + test + + + org.slf4j + slf4j-log4j12 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/pom.xml b/pom.xml index 57e843596f..0257a08184 100644 --- a/pom.xml +++ b/pom.xml @@ -87,12 +87,17 @@ core bagel - examples mllib tools streaming repl assembly + external/twitter + external/kafka + external/flume + external/zeromq + external/mqtt + examples @@ -140,17 +145,6 @@ false - - mqtt-repo - MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases - - true - - - false - - @@ -253,17 +247,6 @@ - - ${akka.group} - akka-zeromq_${scala.binary.version} - ${akka.version} - - - org.jboss.netty - netty - - - it.unimi.dsi fastutil diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4e92d2a162..b7aa73208a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -141,7 +141,7 @@ object SparkBuild extends Build { // also check the local Maven repository ~/.m2 resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), - // For Sonatype publishing + // 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/"), @@ -315,9 +315,6 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", - resolvers ++= Seq( - "Apache repo" at "https://repository.apache.org/content/repositories/releases" - ), libraryDependencies ++= Seq( "commons-io" % "commons-io" % "2.4" ) @@ -360,14 +357,14 @@ object SparkBuild extends Build { } ) - def twitterSettings() = streamingSettings ++ Seq( + def twitterSettings() = sharedSettings ++ Seq( name := "spark-streaming-twitter", libraryDependencies ++= Seq( "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) ) ) - def kafkaSettings() = streamingSettings ++ Seq( + def kafkaSettings() = sharedSettings ++ Seq( name := "spark-streaming-kafka", libraryDependencies ++= Seq( "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), @@ -379,14 +376,14 @@ object SparkBuild extends Build { ) ) - def flumeSettings() = streamingSettings ++ Seq( + def flumeSettings() = sharedSettings ++ Seq( name := "spark-streaming-flume", libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy) ) ) - def zeromqSettings() = streamingSettings ++ Seq( + def zeromqSettings() = sharedSettings ++ Seq( name := "spark-streaming-zeromq", libraryDependencies ++= Seq( "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) @@ -395,11 +392,7 @@ object SparkBuild extends Build { def mqttSettings() = streamingSettings ++ Seq( name := "spark-streaming-mqtt", - resolvers ++= Seq( - "Apache repo" at "https://repository.apache.org/content/repositories/releases" - ), - libraryDependencies ++= Seq( - "org.eclipse.paho" % "mqtt-client" % "0.4.0" - ) + resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"), + libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") ) } diff --git a/streaming/pom.xml b/streaming/pom.xml index e3b6fee9b2..459756912d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -60,59 +60,10 @@ jackson-mapper-asl 1.9.11 - - com.sksamuel.kafka - kafka_${scala.binary.version} - 0.8.0-beta1 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - net.sf.jopt-simple - jopt-simple - - - - - org.apache.flume - flume-ng-sdk - 1.2.0 - - - org.jboss.netty - netty - - - org.xerial.snappy - * - - - - - org.twitter4j - twitter4j-stream - 3.0.3 - - - org.jboss.netty - netty - - - org.scala-lang scala-library - - ${akka.group} - akka-zeromq_${scala.binary.version} - org.scalatest scalatest_${scala.binary.version} @@ -137,11 +88,6 @@ commons-io commons-io - - org.eclipse.paho - mqtt-client - 0.4.0 - target/scala-${scala.binary.version}/classes @@ -151,6 +97,35 @@ org.scalatest scalatest-maven-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + 2.2 + + + + test-jar + + + + test-jar-on-compile + compile + + test-jar + + + + -- cgit v1.2.3 From 94b7a7fe37a4b1459bfdbece2a4162451d6a8ac2 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 18:41:21 +0530 Subject: run-example -> bin/run-example --- README.md | 4 ++-- bin/run-example | 2 +- bin/run-example2.cmd | 2 +- docs/bagel-programming-guide.md | 4 ++-- docs/index.md | 6 +++--- docs/java-programming-guide.md | 4 ++-- docs/running-on-yarn.md | 2 +- docs/scala-programming-guide.md | 4 ++-- docs/streaming-programming-guide.md | 4 ++-- .../org/apache/spark/streaming/examples/JavaKafkaWordCount.java | 2 +- .../scala/org/apache/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../scala/org/apache/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/MQTTWordCount.scala | 4 ++-- .../org/apache/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../apache/spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../spark/streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../spark/streaming/examples/clickstream/PageViewStream.scala | 4 ++-- 19 files changed, 31 insertions(+), 31 deletions(-) (limited to 'examples') diff --git a/README.md b/README.md index 170e964851..7154165ab1 100644 --- a/README.md +++ b/README.md @@ -24,9 +24,9 @@ Once you've built Spark, the easiest way to start using it is the shell: Or, for the Python API, the Python shell (`./pyspark`). Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./run-example `. For example: +To run one of them, use `./bin/run-example `. For example: - ./run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example org.apache.spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. diff --git a/bin/run-example b/bin/run-example index f2699c38a9..a3976beb12 100755 --- a/bin/run-example +++ b/bin/run-example @@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then fi if [ -z "$1" ]; then - echo "Usage: run-example []" >&2 + echo "Usage: bin/run-example []" >&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index 6861334cb0..d9c10545e7 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [^] + echo Usage: bin/run-example ^ [^] goto exit :arg_given diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index de001e6c52..c4f1f6d6ad 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -157,8 +157,8 @@ trait Message[K] { # Where to Go from Here -Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.: +Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.: - ./run-example org.apache.spark.examples.bagel.WikipediaPageRank + ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank Each example program prints usage help when run without any arguments. diff --git a/docs/index.md b/docs/index.md index a5c179c19c..f0990fff73 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE # Running the Examples and Shell Spark comes with several sample programs in the `examples` directory. -To run one of the samples, use `./run-example ` in the top-level Spark directory -(the `run-example` script sets up the appropriate paths and launches that program). -For example, try `./run-example org.apache.spark.examples.SparkPi local`. +To run one of the samples, use `./bin/run-example ` in the top-level Spark directory +(the `bin/run-example` script sets up the appropriate paths and launches that program). +For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. Each example prints usage help when run with no parameters. Note that all of the sample programs take a `` parameter specifying the cluster URL diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 53085cc671..07732fa122 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -190,9 +190,9 @@ We hope to generate documentation with Java-style syntax in the future. Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the -`run-example` script included in Spark; for example: +`bin/run-example` script included in Spark; for example: - ./run-example org.apache.spark.examples.JavaWordCount + ./bin/run-example org.apache.spark.examples.JavaWordCount Each example program prints usage help when run without any arguments. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index be32034b3e..e2b21f9cde 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -107,7 +107,7 @@ For example: SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ - ./run-example org.apache.spark.examples.SparkPi yarn-client + ./bin/run-example org.apache.spark.examples.SparkPi yarn-client SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 16ae15046a..c1ef46a1cd 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -366,9 +366,9 @@ res2: Int = 10 # Where to Go from Here You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website. -In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example: +In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: - ./run-example org.apache.spark.examples.SparkPi + ./bin/run-example org.apache.spark.examples.SparkPi Each example program prints usage help when run without any arguments. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6f97db6a3e..1c9ece6270 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -245,7 +245,7 @@ $ nc -lk 9999 Then, in a different terminal, you can start NetworkWordCount by using {% highlight bash %} -$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 {% endhighlight %} This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen. @@ -283,7 +283,7 @@ Time: 1357008430000 ms -You can find more examples in `/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. +You can find more examples in `/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. # DStream Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec..75b588e4b8 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -41,7 +41,7 @@ import scala.Tuple2; * is the number of threads the kafka consumer should use * * Example: - * `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, + * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, * zoo03 my-consumer-group topic1,topic2 1` */ 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 2402409e6e..3641517934 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 @@ -134,9 +134,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index bc8564b3ba..61be1ce4b1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 570ba4c81a..8dc8a3531a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ff332a0282..ea138f55e8 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -79,9 +79,9 @@ object MQTTPublisher { * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` */ object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index e2487dca5f..ce8df8c502 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index cb30c4edb3..e55d71edfc 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { 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 e83ce78aa5..2948aa7cc4 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 @@ -62,9 +62,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index de70c50473..1a40fdb9a3 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -39,8 +39,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 8282cc9269..968b578487 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ object PageViewStream { def main(args: Array[String]) { -- cgit v1.2.3 From 66d501276b5a066bd9abaa4e284cfad557665948 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 2 Jan 2014 16:17:57 +0000 Subject: Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance --- .../org/apache/spark/network/netty/FileClient.java | 5 ++- .../netty/FileClientChannelInitializer.java | 2 +- .../org/apache/spark/network/netty/FileServer.java | 8 ++-- .../netty/FileServerChannelInitializer.java | 4 +- .../spark/network/netty/FileServerHandler.java | 6 +-- .../apache/spark/network/netty/PathResolver.java | 52 +++++++++++----------- .../java/org/apache/spark/examples/JavaHdfsLR.java | 32 +++++++------ .../java/org/apache/spark/examples/JavaKMeans.java | 26 +++++++---- .../org/apache/spark/examples/JavaLogQuery.java | 23 +++++----- .../org/apache/spark/examples/JavaPageRank.java | 14 ++++-- .../org/apache/spark/examples/JavaSparkPi.java | 11 +++-- .../java/org/apache/spark/examples/JavaTC.java | 19 +++++--- .../org/apache/spark/examples/JavaWordCount.java | 13 +++++- .../org/apache/spark/mllib/examples/JavaALS.java | 21 ++++++--- .../apache/spark/mllib/examples/JavaKMeans.java | 19 +++++--- .../org/apache/spark/mllib/examples/JavaLR.java | 22 +++++---- .../streaming/examples/JavaFlumeEventCount.java | 5 ++- .../streaming/examples/JavaKafkaWordCount.java | 16 ++++--- .../streaming/examples/JavaNetworkWordCount.java | 15 +++++-- .../spark/streaming/examples/JavaQueueStream.java | 11 +++-- 20 files changed, 203 insertions(+), 121 deletions(-) (limited to 'examples') 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 46d61503bc..d2d778b756 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 @@ -31,7 +31,8 @@ import java.util.concurrent.TimeUnit; class FileClient { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); + private final FileClientHandler handler; private Channel channel = null; private Bootstrap bootstrap = null; @@ -39,7 +40,7 @@ class FileClient { private final int connectTimeout; private final int sendTimeout = 60; // 1 min - public FileClient(FileClientHandler handler, int connectTimeout) { + FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; this.connectTimeout = connectTimeout; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index fb61be1c12..264cf97d02 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -25,7 +25,7 @@ class FileClientChannelInitializer extends ChannelInitializer { private final FileClientHandler fhandler; - public FileClientChannelInitializer(FileClientHandler handler) { + FileClientChannelInitializer(FileClientHandler handler) { fhandler = handler; } 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 aea7534459..c93425e278 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 @@ -33,15 +33,14 @@ import org.slf4j.LoggerFactory; */ class FileServer { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName()); private EventLoopGroup bossGroup = null; private EventLoopGroup workerGroup = null; private ChannelFuture channelFuture = null; private int port = 0; - private Thread blockingThread = null; - public FileServer(PathResolver pResolver, int port) { + FileServer(PathResolver pResolver, int port) { InetSocketAddress addr = new InetSocketAddress(port); // Configure the server. @@ -70,7 +69,8 @@ class FileServer { * Start the file server asynchronously in a new thread. */ public void start() { - blockingThread = new Thread() { + Thread blockingThread = new Thread() { + @Override public void run() { try { channelFuture.channel().closeFuture().sync(); diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index 3f15ff898f..46efec8f8d 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -25,9 +25,9 @@ import io.netty.handler.codec.string.StringDecoder; class FileServerChannelInitializer extends ChannelInitializer { - PathResolver pResolver; + private final PathResolver pResolver; - public FileServerChannelInitializer(PathResolver pResolver) { + FileServerChannelInitializer(PathResolver pResolver) { this.pResolver = pResolver; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index e2d9391b4c..3ac045f944 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -31,11 +31,11 @@ import org.slf4j.LoggerFactory; class FileServerHandler extends SimpleChannelInboundHandler { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName()); private final PathResolver pResolver; - public FileServerHandler(PathResolver pResolver){ + FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; } @@ -61,7 +61,7 @@ class FileServerHandler extends SimpleChannelInboundHandler { ctx.flush(); return; } - int len = new Long(length).intValue(); + int len = (int) length; ctx.write((new FileHeader(len, blockId)).buffer()); try { ctx.write(new DefaultFileRegion(new FileInputStream(file) diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java index 9f7ced44cf..7ad8d03efb 100755 --- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java +++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java @@ -1,26 +1,26 @@ -/* - * 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.network.netty; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; - -public interface PathResolver { - /** Get the file segment in which the given block resides. */ - public FileSegment getBlockLocation(BlockId blockId); -} +/* + * 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.network.netty; + +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.FileSegment; + +public interface PathResolver { + /** Get the file segment in which the given block resides. */ + FileSegment getBlockLocation(BlockId blockId); +} diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c..9f0e3412a6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -24,19 +24,22 @@ import org.apache.spark.api.java.function.Function2; import java.io.Serializable; import java.util.Arrays; -import java.util.StringTokenizer; import java.util.Random; +import java.util.regex.Pattern; /** * Logistic regression based classification. */ -public class JavaHdfsLR { +public final class JavaHdfsLR { - static int D = 10; // Number of dimensions - static Random rand = new Random(42); + private static final int D = 10; // Number of dimensions + private static final Random rand = new Random(42); + + private JavaHdfsLR() { + } static class DataPoint implements Serializable { - public DataPoint(double[] x, double y) { + DataPoint(double[] x, double y) { this.x = x; this.y = y; } @@ -46,20 +49,22 @@ public class JavaHdfsLR { } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public DataPoint call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - double y = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double y = Double.parseDouble(tok[0]); double[] x = new double[D]; - int i = 0; - while (i < D) { - x[i] = Double.parseDouble(tok.nextToken()); - i += 1; + for (int i = 0; i < D; i++) { + x[i] = Double.parseDouble(tok[i+1]); } return new DataPoint(x, y); } } static class VectorSum extends Function2 { + @Override public double[] call(double[] a, double[] b) { double[] result = new double[D]; for (int j = 0; j < D; j++) { @@ -70,12 +75,13 @@ public class JavaHdfsLR { } static class ComputeGradient extends Function { - double[] weights; + private final double[] weights; - public ComputeGradient(double[] weights) { + ComputeGradient(double[] weights) { this.weights = weights; } + @Override public double[] call(DataPoint p) { double[] gradient = new double[D]; for (int i = 0; i < D; i++) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eae..1671d0cdc8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -27,19 +27,27 @@ import org.apache.spark.util.Vector; import java.util.List; import java.util.Map; +import java.util.regex.Pattern; /** * K-means clustering using Java API. */ -public class JavaKMeans { +public final class JavaKMeans { + + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKMeans() { + } /** Parses numbers split by whitespace to a vector */ static Vector parseVector(String line) { - String[] splits = line.split(" "); + String[] splits = SPACE.split(line); double[] data = new double[splits.length]; int i = 0; - for (String s : splits) - data[i] = Double.parseDouble(splits[i++]); + for (String s : splits) { + data[i] = Double.parseDouble(s); + i++; + } return new Vector(data); } @@ -82,7 +90,7 @@ public class JavaKMeans { JavaRDD data = sc.textFile(path).map( new Function() { @Override - public Vector call(String line) throws Exception { + public Vector call(String line) { return parseVector(line); } } @@ -96,7 +104,7 @@ public class JavaKMeans { JavaPairRDD closest = data.map( new PairFunction() { @Override - public Tuple2 call(Vector vector) throws Exception { + public Tuple2 call(Vector vector) { return new Tuple2( closestPoint(vector, centroids), vector); } @@ -107,7 +115,8 @@ public class JavaKMeans { JavaPairRDD> pointsGroup = closest.groupByKey(); Map newCentroids = pointsGroup.mapValues( new Function, Vector>() { - public Vector call(List ps) throws Exception { + @Override + public Vector call(List ps) { return average(ps); } }).collectAsMap(); @@ -122,8 +131,9 @@ public class JavaKMeans { } while (tempDist > convergeDist); System.out.println("Final centers:"); - for (Vector c : centroids) + for (Vector c : centroids) { System.out.println(c); + } System.exit(0); 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 407cd7ccfa..1ce53fe403 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -35,9 +35,9 @@ import java.util.regex.Pattern; /** * Executes a roll up-style query against Apache logs. */ -public class JavaLogQuery { +public final class JavaLogQuery { - public static List exampleApacheLogs = Lists.newArrayList( + public static final List exampleApacheLogs = Lists.newArrayList( "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + @@ -51,14 +51,17 @@ public class JavaLogQuery { "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + "0 73.23.2.15 images.com 1358492557 - Whatup"); - public static Pattern apacheLogRegex = Pattern.compile( + public static final Pattern apacheLogRegex = Pattern.compile( "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + private JavaLogQuery() { + } + /** Tracks the total query count and number of aggregate bytes for a particular group. */ public static class Stats implements Serializable { - private int count; - private int numBytes; + private final int count; + private final int numBytes; public Stats(int count, int numBytes) { this.count = count; @@ -92,12 +95,12 @@ public class JavaLogQuery { if (m.find()) { int bytes = Integer.parseInt(m.group(7)); return new Stats(1, bytes); - } - else + } else { return new Stats(1, 0); + } } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { if (args.length == 0) { System.err.println("Usage: JavaLogQuery [logFile]"); System.exit(1); @@ -110,14 +113,14 @@ public class JavaLogQuery { JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { @Override - public Tuple2, Stats> call(String s) throws Exception { + public Tuple2, Stats> call(String s) { return new Tuple2, Stats>(extractKey(s), extractStats(s)); } }); JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { @Override - public Stats call(Stats stats, Stats stats2) throws Exception { + public Stats call(Stats stats, Stats stats2) { return stats.merge(stats2); } }); 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 89aed8f279..447ba93bd6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -28,6 +28,7 @@ import org.apache.spark.api.java.function.PairFunction; import java.util.List; import java.util.ArrayList; +import java.util.regex.Pattern; /** * Computes the PageRank of URLs from an input file. Input file should @@ -38,7 +39,12 @@ import java.util.ArrayList; * ... * where URL and their neighbors are separated by space(s). */ -public class JavaPageRank { +public final class JavaPageRank { + private static final Pattern SPACES = Pattern.compile("\\s+"); + + private JavaPageRank() { + } + private static class Sum extends Function2 { @Override public Double call(Double a, Double b) { @@ -66,7 +72,7 @@ public class JavaPageRank { JavaPairRDD> links = lines.map(new PairFunction() { @Override public Tuple2 call(String s) { - String[] parts = s.split("\\s+"); + String[] parts = SPACES.split(s); return new Tuple2(parts[0], parts[1]); } }).distinct().groupByKey().cache(); @@ -74,7 +80,7 @@ public class JavaPageRank { // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) throws Exception { + public Double call(List rs) { return 1.0; } }); @@ -97,7 +103,7 @@ public class JavaPageRank { // Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(new Sum()).mapValues(new Function() { @Override - public Double call(Double sum) throws Exception { + public Double call(Double sum) { return 0.15 + sum * 0.85; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5..d2a2a1db7c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -26,8 +26,10 @@ import java.util.ArrayList; import java.util.List; /** Computes an approximation to pi */ -public class JavaSparkPi { +public final class JavaSparkPi { + private JavaSparkPi() { + } public static void main(String[] args) throws Exception { if (args.length == 0) { @@ -41,21 +43,22 @@ public class JavaSparkPi { int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; List l = new ArrayList(n); - for (int i = 0; i < n; i++) + for (int i = 0; i < n; i++) { l.add(i); + } JavaRDD dataSet = jsc.parallelize(l, slices); int count = dataSet.map(new Function() { @Override - public Integer call(Integer integer) throws Exception { + public Integer call(Integer integer) { double x = Math.random() * 2 - 1; double y = Math.random() * 2 - 1; return (x * x + y * y < 1) ? 1 : 0; } }).reduce(new Function2() { @Override - public Integer call(Integer integer, Integer integer2) throws Exception { + public Integer call(Integer integer, Integer integer2) { return integer + integer2; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b77..e61b9c4f0e 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -31,11 +31,14 @@ import java.util.Set; /** * Transitive closure on a graph, implemented in Java. */ -public class JavaTC { +public final class JavaTC { - static int numEdges = 200; - static int numVertices = 100; - static Random rand = new Random(42); + private static final int numEdges = 200; + private static final int numVertices = 100; + private static final Random rand = new Random(42); + + private JavaTC() { + } static List> generateGraph() { Set> edges = new HashSet>(numEdges); @@ -43,15 +46,18 @@ public class JavaTC { int from = rand.nextInt(numVertices); int to = rand.nextInt(numVertices); Tuple2 e = new Tuple2(from, to); - if (from != to) edges.add(e); + if (from != to) { + edges.add(e); + } } return new ArrayList>(edges); } static class ProjectFn extends PairFunction>, Integer, Integer> { - static ProjectFn INSTANCE = new ProjectFn(); + static final ProjectFn INSTANCE = new ProjectFn(); + @Override public Tuple2 call(Tuple2> triple) { return new Tuple2(triple._2()._2(), triple._2()._1()); } @@ -76,6 +82,7 @@ public class JavaTC { // Because join() joins on keys, the edges are stored in reversed order. JavaPairRDD edges = tc.map( new PairFunction, Integer, Integer>() { + @Override public Tuple2 call(Tuple2 e) { return new Tuple2(e._2(), e._1()); } 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 bd6383e13d..ed4e9b49d0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -27,8 +27,14 @@ import org.apache.spark.api.java.function.PairFunction; import java.util.Arrays; import java.util.List; +import java.util.regex.Pattern; + +public final class JavaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaWordCount() { + } -public class JavaWordCount { public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaWordCount "); @@ -40,18 +46,21 @@ public class JavaWordCount { JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { + @Override public Iterable call(String s) { - return Arrays.asList(s.split(" ")); + return Arrays.asList(SPACE.split(s)); } }); JavaPairRDD ones = words.map(new PairFunction() { + @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); JavaPairRDD counts = ones.reduceByKey(new Function2() { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } 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 45a0d237da..b33e648147 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 @@ -26,28 +26,35 @@ import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; import org.apache.spark.mllib.recommendation.Rating; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; import scala.Tuple2; /** * Example using MLLib ALS from Java. */ -public class JavaALS { +public final class JavaALS { + + private JavaALS() { + } static class ParseRating extends Function { + private static final Pattern COMMA = Pattern.compile(","); + + @Override public Rating call(String line) { - StringTokenizer tok = new StringTokenizer(line, ","); - int x = Integer.parseInt(tok.nextToken()); - int y = Integer.parseInt(tok.nextToken()); - double rating = Double.parseDouble(tok.nextToken()); + String[] tok = COMMA.split(line); + int x = Integer.parseInt(tok[0]); + int y = Integer.parseInt(tok[1]); + double rating = Double.parseDouble(tok[2]); return new Rating(x, y, rating); } } static class FeaturesToString extends Function, String> { + @Override public String call(Tuple2 element) { - return element._1().toString() + "," + Arrays.toString(element._2()); + return element._1() + "," + Arrays.toString(element._2()); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9..a9db04d525 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -25,20 +25,25 @@ import org.apache.spark.mllib.clustering.KMeans; import org.apache.spark.mllib.clustering.KMeansModel; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Example using MLLib KMeans from Java. */ -public class JavaKMeans { +public final class JavaKMeans { + + private JavaKMeans() { + } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public double[] call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - int numTokens = tok.countTokens(); - double[] point = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - point[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double[] point = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + point[i] = Double.parseDouble(tok[i]); } return point; } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6..56341315bf 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -27,22 +27,28 @@ import org.apache.spark.mllib.classification.LogisticRegressionModel; import org.apache.spark.mllib.regression.LabeledPoint; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Logistic regression based classification using ML Lib. */ -public class JavaLR { +public final class JavaLR { + + private JavaLR() { + } static class ParsePoint extends Function { + private static final Pattern COMMA = Pattern.compile(","); + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public LabeledPoint call(String line) { - String[] parts = line.split(","); + String[] parts = COMMA.split(line); double y = Double.parseDouble(parts[0]); - StringTokenizer tok = new StringTokenizer(parts[1], " "); - int numTokens = tok.countTokens(); - double[] x = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - x[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(parts[1]); + double[] x = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + x[i] = Double.parseDouble(tok[i]); } return new LabeledPoint(y, x); } diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f..bd0bbb56ff 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -36,7 +36,10 @@ import org.apache.spark.streaming.dstream.SparkFlumeEvent; * creates a server and listens for flume events. * is the port the Flume receiver will listen on. */ -public class JavaFlumeEventCount { +public final class JavaFlumeEventCount { + private JavaFlumeEventCount() { + } + public static void main(String[] args) { if (args.length != 3) { System.err.println("Usage: JavaFlumeEventCount "); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec..17eb871908 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples; import java.util.Map; import java.util.HashMap; +import java.util.regex.Pattern; import com.google.common.collect.Lists; import org.apache.spark.api.java.function.FlatMapFunction; @@ -45,7 +46,12 @@ import scala.Tuple2; * zoo03 my-consumer-group topic1,topic2 1` */ -public class JavaKafkaWordCount { +public final class JavaKafkaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKafkaWordCount() { + } + public static void main(String[] args) { if (args.length < 5) { System.err.println("Usage: KafkaWordCount "); @@ -67,7 +73,7 @@ public class JavaKafkaWordCount { JavaDStream lines = messages.map(new Function, String>() { @Override - public String call(Tuple2 tuple2) throws Exception { + public String call(Tuple2 tuple2) { return tuple2._2(); } }); @@ -75,19 +81,19 @@ public class JavaKafkaWordCount { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199b..fb090cc262 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -27,6 +27,8 @@ 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 java.util.regex.Pattern; + /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: NetworkWordCount @@ -38,7 +40,12 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; * and then run the example * `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` */ -public class JavaNetworkWordCount { +public final class JavaNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaNetworkWordCount() { + } + public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: NetworkWordCount \n" + @@ -56,18 +63,18 @@ public class JavaNetworkWordCount { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1..6be967237c 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -31,8 +31,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; -public class JavaQueueStream { - public static void main(String[] args) throws InterruptedException { +public final class JavaQueueStream { + private JavaQueueStream() { + } + + public static void main(String[] args) throws Exception { if (args.length < 1) { System.err.println("Usage: JavaQueueStream "); System.exit(1); @@ -62,14 +65,14 @@ public class JavaQueueStream { JavaPairDStream mappedStream = inputStream.map( new PairFunction() { @Override - public Tuple2 call(Integer i) throws Exception { + public Tuple2 call(Integer i) { return new Tuple2(i % 10, 1); } }); JavaPairDStream reducedStream = mappedStream.reduceByKey( new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); -- cgit v1.2.3 From 79f52809c836d08023aa5ca99a467d3a311a7359 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 11:43:00 -0800 Subject: Removing SPARK_EXAMPLES_JAR in the code --- .../apache/spark/api/java/JavaSparkContext.scala | 6 ++++++ .../java/org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../java/org/apache/spark/examples/JavaKMeans.java | 2 +- .../org/apache/spark/examples/JavaLogQuery.java | 2 +- .../org/apache/spark/examples/JavaPageRank.java | 3 ++- .../org/apache/spark/examples/JavaSparkPi.java | 2 +- .../java/org/apache/spark/examples/JavaTC.java | 2 +- .../org/apache/spark/examples/JavaWordCount.java | 2 +- .../org/apache/spark/mllib/examples/JavaALS.java | 2 +- .../apache/spark/mllib/examples/JavaKMeans.java | 2 +- .../org/apache/spark/mllib/examples/JavaLR.java | 2 +- .../streaming/examples/JavaFlumeEventCount.java | 3 ++- .../streaming/examples/JavaKafkaWordCount.java | 3 ++- .../streaming/examples/JavaNetworkWordCount.java | 3 ++- .../spark/streaming/examples/JavaQueueStream.java | 2 +- .../org/apache/spark/examples/BroadcastTest.scala | 2 +- .../spark/examples/ExceptionHandlingTest.scala | 2 +- .../org/apache/spark/examples/GroupByTest.scala | 2 +- .../org/apache/spark/examples/HBaseTest.scala | 2 +- .../scala/org/apache/spark/examples/HdfsTest.scala | 2 +- .../scala/org/apache/spark/examples/LogQuery.scala | 2 +- .../apache/spark/examples/MultiBroadcastTest.scala | 2 +- .../spark/examples/SimpleSkewedGroupByTest.scala | 2 +- .../apache/spark/examples/SkewedGroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/SparkALS.scala | 2 +- .../org/apache/spark/examples/SparkHdfsLR.scala | 2 +- .../org/apache/spark/examples/SparkKMeans.scala | 2 +- .../scala/org/apache/spark/examples/SparkLR.scala | 2 +- .../org/apache/spark/examples/SparkPageRank.scala | 2 +- .../scala/org/apache/spark/examples/SparkPi.scala | 2 +- .../scala/org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/FlumeEventCount.scala | 2 +- .../spark/streaming/examples/HdfsWordCount.scala | 2 +- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/NetworkWordCount.scala | 2 +- .../spark/streaming/examples/QueueStream.scala | 2 +- .../spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../examples/StatefulNetworkWordCount.scala | 2 +- .../streaming/examples/TwitterAlgebirdCMS.scala | 2 +- .../streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../streaming/examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../apache/spark/streaming/StreamingContext.scala | 6 ++++++ .../streaming/api/java/JavaStreamingContext.scala | 25 +++++++++++++--------- 47 files changed, 75 insertions(+), 54 deletions(-) (limited to 'examples') diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 5be5317f40..e93b10fd7e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -431,4 +431,10 @@ object JavaSparkContext { implicit def fromSparkContext(sc: SparkContext): JavaSparkContext = new JavaSparkContext(sc) implicit def toSparkContext(jsc: JavaSparkContext): SparkContext = jsc.sc + + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c..12f3355bc4 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -106,7 +106,7 @@ public class JavaHdfsLR { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[2]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eae..63465a3bbf 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -74,7 +74,7 @@ public class JavaKMeans { System.exit(1); } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); String path = args[1]; int K = Integer.parseInt(args[2]); double convergeDist = Double.parseDouble(args[3]); 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 407cd7ccfa..74e4d9291a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -104,7 +104,7 @@ public class JavaLogQuery { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class)); JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); 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 89aed8f279..f774f6a04e 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,6 +17,7 @@ package org.apache.spark.examples; +import org.apache.spark.SparkContext; import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -53,7 +54,7 @@ public class JavaPageRank { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class)); // Loads in input file. It should be in format of: // URL neighbor URL diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5..5558ab7c03 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -36,7 +36,7 @@ public class JavaSparkPi { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class)); int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b77..99e6ba347c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -64,7 +64,7 @@ public class JavaTC { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class)); Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); 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 bd6383e13d..8a071caf13 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -36,7 +36,7 @@ public class JavaWordCount { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class)); JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { 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 45a0d237da..5e1a77baaa 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 @@ -68,7 +68,7 @@ public class JavaALS { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD ratings = lines.map(new ParseRating()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9..1f12f518a0 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -62,7 +62,7 @@ public class JavaKMeans { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6..593e4df111 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -59,7 +59,7 @@ public class JavaLR { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); double stepSize = Double.parseDouble(args[2]); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f..64ac72474b 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -50,7 +50,8 @@ public class JavaFlumeEventCount { Duration batchInterval = new Duration(2000); JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); JavaDStream flumeStream = sc.flumeStream("localhost", port); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 75b588e4b8..0a56e7abdf 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -54,7 +54,8 @@ public class JavaKafkaWordCount { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", - new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(2000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); int numThreads = Integer.parseInt(args[4]); Map topicMap = new HashMap(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199b..ec6f6a8c56 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -48,7 +48,8 @@ public class JavaNetworkWordCount { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1..4b9fd52713 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -40,7 +40,7 @@ public class JavaQueueStream { // Create the context JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class)); // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index a119980992..0097dade19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -33,7 +33,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 92eb96bd8e..b3eb611dd2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 42c2e0e8e1..39752fdd0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -34,7 +34,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index efe2e93b0d..65d67356be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormat object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index d6a88d3032..c3597d94a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 17ff3ce764..bddb54b39c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -45,7 +45,7 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index e1afc29f9a..4aef04fc06 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 37ddfb5db7..73b0e216ca 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 9c954b2b5b..31c6d108f3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -34,7 +34,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 814944ba1c..30c86d83e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 86dd9ca1b3..ff72532db1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -54,7 +54,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)))) val lines = sc.textFile(inputPath) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index bc2db39c12..8c99025eaa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -55,7 +55,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 9ed9fe4d76..c54a55bdb4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -49,7 +49,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index a508c0df57..d203f4d20e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -38,7 +38,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index a689e5a360..e5a09ecec0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 8543ce0e32..24e8afa26b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -46,7 +46,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() 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 3641517934..546495357f 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 @@ -151,7 +151,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 9f6e163454..5ef1928294 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -48,7 +48,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a flume stream val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 61be1ce4b1..1486d77d8a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -40,7 +40,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 8dc8a3531a..172091be2e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -48,7 +48,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ea138f55e8..2d02ef77c0 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -96,7 +96,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + StreamingContext.jarOfClass(this.getClass)) val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index ce8df8c502..74d76ec26c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -41,7 +41,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index fad512eeba..9d640e716b 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -33,7 +33,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index 0b45c30d20..c0706d0724 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -49,7 +49,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index e55d71edfc..f43c8ab61d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -49,7 +49,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 35b6329ab3..9d21d3178f 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -60,7 +60,7 @@ object TwitterAlgebirdCMS { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 8bfde2a829..5111e6f62a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -49,7 +49,7 @@ object TwitterAlgebirdHLL { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 27aa6b14bf..7a3df687b7 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -38,7 +38,7 @@ object TwitterPopularTags { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) 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 2948aa7cc4..89d3042123 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 @@ -78,7 +78,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 968b578487..0569846f18 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -42,7 +42,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) 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 304986f187..b3a7cf08b9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -595,6 +595,12 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls) + protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second batch intervals. 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 b79173c6aa..7dec4b3ad7 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 @@ -17,29 +17,27 @@ package org.apache.spark.streaming.api.java -import java.lang.{Integer => JInt} import java.io.InputStream -import java.util.{Map => JMap, List => JList} +import java.lang.{Integer => JInt} +import java.util.{List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import akka.actor.{Props, SupervisorStrategy} +import akka.util.ByteString +import akka.zeromq.Subscribe import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.zeromq.Subscribe -import akka.util.ByteString - import twitter4j.auth.Authorization +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.SparkConf import org.apache.spark.streaming.scheduler.StreamingListener /** @@ -716,5 +714,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Sstops the execution of the streams. */ def stop() = ssc.stop() +} +object JavaStreamingContext { + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } -- cgit v1.2.3 From d0fd3b9ad238294346eb3465c489eabd41fb2380 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 6 Jan 2014 01:47:53 -0800 Subject: Changed JavaStreamingContextWith*** to ***Function in streaming.api.java.*** package. Also fixed packages of Flume and MQTT tests. --- .../streaming/examples/JavaFlumeEventCount.java | 6 +- .../streaming/examples/JavaKafkaWordCount.java | 6 +- .../streaming/api/java/flume/FlumeFunctions.scala | 48 +++++++++ .../flume/JavaStreamingContextWithFlume.scala | 48 --------- .../flume/src/test/java/JavaFlumeStreamSuite.java | 38 ------- .../streaming/flume/JavaFlumeStreamSuite.java | 35 +++++++ .../streaming/api/java/kafka/KafkaFunctions.scala | 107 ++++++++++++++++++++ .../kafka/JavaStreamingContextWithKafka.scala | 107 -------------------- .../streaming/kafka/JavaKafkaStreamSuite.java | 15 ++- .../mqtt/JavaStreamingContextWithMQTT.scala | 59 ----------- .../spark/spark/streaming/mqtt/MQTTFunctions.scala | 43 -------- .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 --------------------- .../spark/spark/streaming/mqtt/package.scala | 24 ----- .../streaming/api/java/mqtt/MQTTFunctions.scala | 59 +++++++++++ .../spark/streaming/mqtt/MQTTFunctions.scala | 43 ++++++++ .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 +++++++++++++++++++++ .../org/apache/spark/streaming/mqtt/package.scala | 24 +++++ .../spark/streaming/mqtt/JavaMQTTStreamSuite.java | 10 +- .../api/java/twitter/TwitterFunctions.scala | 99 +++++++++++++++++++ .../twitter/JavaStreamingContextWithTwitter.scala | 99 ------------------- .../streaming/twitter/JavaTwitterStreamSuite.java | 19 ++-- .../api/java/zeromq/ZeroMQFunctions.scala | 102 +++++++++++++++++++ .../zeromq/JavaStreamingContextWithZeroMQ.scala | 102 ------------------- .../streaming/zeromq/JavaZeroMQStreamSuite.java | 12 +-- 24 files changed, 658 insertions(+), 667 deletions(-) create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala delete mode 100644 external/flume/src/test/java/JavaFlumeStreamSuite.java create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala (limited to 'examples') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index e53c4f9e83..64832a9721 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -20,7 +20,7 @@ package org.apache.spark.streaming.examples; import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; +import org.apache.spark.streaming.api.java.flume.FlumeFunctions; import org.apache.spark.streaming.flume.SparkFlumeEvent; /** @@ -52,8 +52,8 @@ public class JavaFlumeEventCount { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); - JavaDStream flumeStream = sscWithFlume.flumeStream("localhost", port); + FlumeFunctions flumeFunc = new FlumeFunctions(ssc); + JavaDStream flumeStream = flumeFunc.flumeStream("localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index de0420ca83..207ce8cd4f 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,7 +29,7 @@ import org.apache.spark.streaming.Duration; 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.kafka.JavaStreamingContextWithKafka; +import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import scala.Tuple2; /** @@ -64,8 +64,8 @@ public class JavaKafkaWordCount { topicMap.put(topic, numThreads); } - JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); - JavaPairDStream messages = sscWithKafka.kafkaStream(args[1], args[2], topicMap); + KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); + JavaPairDStream messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala new file mode 100644 index 0000000000..3347d19796 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala @@ -0,0 +1,48 @@ +/* + * 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.api.java.flume + +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.flume._ +import org.apache.spark.storage.StorageLevel + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Flume input streams. + */ +class FlumeFunctions(javaStreamingContext: JavaStreamingContext) { + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + */ + def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { + javaStreamingContext.ssc.flumeStream(hostname, port) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): + JavaDStream[SparkFlumeEvent] = { + javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel) + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala deleted file mode 100644 index 4e66ae3535..0000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala +++ /dev/null @@ -1,48 +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.flume - -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.storage.StorageLevel - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Flume input streams. - */ -class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): - JavaDStream[SparkFlumeEvent] = { - ssc.flumeStream(hostname, port, storageLevel) - } -} diff --git a/external/flume/src/test/java/JavaFlumeStreamSuite.java b/external/flume/src/test/java/JavaFlumeStreamSuite.java deleted file mode 100644 index deffc78c4c..0000000000 --- a/external/flume/src/test/java/JavaFlumeStreamSuite.java +++ /dev/null @@ -1,38 +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. - */ - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume; -import org.apache.spark.streaming.flume.SparkFlumeEvent; -import org.junit.Test; - -public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { - @Test - public void testFlumeStream() { - JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc); - - // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithFlume.flumeStream("localhost", 12345); - JavaDStream test2 = sscWithFlume.flumeStream("localhost", 12345, - StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithFlume.socketTextStream("localhost", 9999); - } -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java new file mode 100644 index 0000000000..5930fee925 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -0,0 +1,35 @@ +package org.apache.spark.streaming.flume;/* + * 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. + */ + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.flume.FlumeFunctions; +import org.apache.spark.streaming.flume.SparkFlumeEvent; +import org.junit.Test; + +public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + FlumeFunctions flumeFunc = new FlumeFunctions(ssc); + + // tests the API, does not actually test data receiving + JavaDStream test1 = flumeFunc.flumeStream("localhost", 12345); + JavaDStream test2 = flumeFunc.flumeStream("localhost", 12345, + StorageLevel.MEMORY_AND_DISK_SER_2()); + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala new file mode 100644 index 0000000000..491331bb37 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala @@ -0,0 +1,107 @@ +/* + * 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.api.java.kafka + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import java.lang.{Integer => JInt} +import java.util.{Map => JMap} + +import kafka.serializer.Decoder + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} +import org.apache.spark.streaming.kafka._ + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Kafka input streams. + */ +class KafkaFunctions(javaStreamingContext: JavaStreamingContext) { + + /** + * Create an input stream that pulls messages form 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 + * in its own thread. + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages form 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 + * in its own thread. + * @param storageLevel RDD storage level. + * + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param keyTypeClass Key type of RDD + * @param valueTypeClass value type of RDD + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. Defaults to memory-only + */ + def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val valueCmt: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + + implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] + implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + + javaStreamingContext.ssc.kafkaStream[K, V, U, T]( + kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala deleted file mode 100644 index ab0e8a6c8d..0000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala +++ /dev/null @@ -1,107 +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.kafka - -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - -import java.lang.{Integer => JInt} -import java.util.{Map => JMap} - -import kafka.serializer.Decoder - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Kafka input streams. - */ -class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { - - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - * @param storageLevel RDD storage level. - * - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. Defaults to memory-only - */ - def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] - - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] - - ssc.kafkaStream[K, V, U, T]( - kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } -} diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 66236df662..fdea96e506 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,6 +18,8 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; + +import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; @@ -31,21 +33,18 @@ public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc); + KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); // tests the API, does not actually test data receiving - JavaPairDStream test1 = sscWithKafka.kafkaStream("localhost:12345", "group", topics); - JavaPairDStream test2 = sscWithKafka.kafkaStream("localhost:12345", "group", topics, + JavaPairDStream test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics); + JavaPairDStream test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); - kafkaParams.put("zookeeper.connect","localhost:12345"); + kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = sscWithKafka.kafkaStream( + JavaPairDStream test3 = kafkaFunc.kafkaStream( String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithKafka.socketTextStream("localhost", 9999); } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala deleted file mode 100644 index d814da0f0d..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.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.mqtt - -import scala.reflect.ClassTag - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating MQTT input streams. - */ -class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - */ - def mqttStream( - brokerUrl: String, - topic: String - ): JavaDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.mqttStream(brokerUrl, topic) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): JavaDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - ssc.mqttStream(brokerUrl, topic, storageLevel) - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala deleted file mode 100644 index 86f4e9c724..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala +++ /dev/null @@ -1,43 +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.mqtt - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions. - */ -class MQTTFunctions(ssc: StreamingContext) { - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala deleted file mode 100644 index c8987a3ee0..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala +++ /dev/null @@ -1,110 +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.mqtt - -import scala.collection.Map -import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ -import scala.reflect.ClassTag - -import java.util.Properties -import java.util.concurrent.Executors -import java.io.IOException - -import org.eclipse.paho.client.mqttv3.MqttCallback -import org.eclipse.paho.client.mqttv3.MqttClient -import org.eclipse.paho.client.mqttv3.MqttClientPersistence -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken -import org.eclipse.paho.client.mqttv3.MqttException -import org.eclipse.paho.client.mqttv3.MqttMessage -import org.eclipse.paho.client.mqttv3.MqttTopic - -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ - -/** - * Input stream that subscribe messages from a Mqtt Broker. - * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ - * @param brokerUrl Url of remote mqtt publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - -private[streaming] -class MQTTInputDStream[T: ClassTag]( - @transient ssc_ : StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[T] = { - new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] - } -} - -private[streaming] -class MQTTReceiver(brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) - - def onStop() { - blockGenerator.stop() - } - - def onStart() { - - blockGenerator.start() - - // Set up persistence for messages - var peristance: MqttClientPersistence = new MemoryPersistence() - - // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance) - - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - - // Callback automatically triggers as and when new message arrives on specified topic - var callback: MqttCallback = new MqttCallback() { - - // Handles Mqtt message - override def messageArrived(arg0: String, arg1: MqttMessage) { - blockGenerator += new String(arg1.getPayload()) - } - - override def deliveryComplete(arg0: IMqttDeliveryToken) { - } - - override def connectionLost(arg0: Throwable) { - logInfo("Connection lost " + arg0) - } - } - - // Set up callback for MqttClient - client.setCallback(callback) - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala deleted file mode 100644 index 28a944f57e..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala +++ /dev/null @@ -1,24 +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 - -package object mqtt { - implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc) -} - - diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala new file mode 100644 index 0000000000..72124956fc --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala @@ -0,0 +1,59 @@ +/* + * 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.api.java.mqtt + +import scala.reflect.ClassTag + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.mqtt._ + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating MQTT input streams. + */ +class MQTTFunctions(javaStreamingContext: JavaStreamingContext) { + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + */ + def mqttStream( + brokerUrl: String, + topic: String + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + javaStreamingContext.ssc.mqttStream(brokerUrl, topic) + } + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. + */ + def mqttStream( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel) + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala new file mode 100644 index 0000000000..86f4e9c724 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala @@ -0,0 +1,43 @@ +/* + * 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.mqtt + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ + +/** + * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]] + * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions. + */ +class MQTTFunctions(ssc: StreamingContext) { + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param brokerUrl Url of remote MQTT publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def mqttStream( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[String] = { + val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala new file mode 100644 index 0000000000..c8987a3ee0 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -0,0 +1,110 @@ +/* + * 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.mqtt + +import scala.collection.Map +import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import java.util.Properties +import java.util.concurrent.Executors +import java.io.IOException + +import org.eclipse.paho.client.mqttv3.MqttCallback +import org.eclipse.paho.client.mqttv3.MqttClient +import org.eclipse.paho.client.mqttv3.MqttClientPersistence +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken +import org.eclipse.paho.client.mqttv3.MqttException +import org.eclipse.paho.client.mqttv3.MqttMessage +import org.eclipse.paho.client.mqttv3.MqttTopic + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ + +/** + * Input stream that subscribe messages from a Mqtt Broker. + * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ + * @param brokerUrl Url of remote mqtt publisher + * @param topic topic name to subscribe to + * @param storageLevel RDD storage level. + */ + +private[streaming] +class MQTTInputDStream[T: ClassTag]( + @transient ssc_ : StreamingContext, + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends NetworkInputDStream[T](ssc_) with Logging { + + def getReceiver(): NetworkReceiver[T] = { + new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] + } +} + +private[streaming] +class MQTTReceiver(brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends NetworkReceiver[Any] { + lazy protected val blockGenerator = new BlockGenerator(storageLevel) + + def onStop() { + blockGenerator.stop() + } + + def onStart() { + + blockGenerator.start() + + // Set up persistence for messages + var peristance: MqttClientPersistence = new MemoryPersistence() + + // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance + var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance) + + // Connect to MqttBroker + client.connect() + + // Subscribe to Mqtt topic + client.subscribe(topic) + + // Callback automatically triggers as and when new message arrives on specified topic + var callback: MqttCallback = new MqttCallback() { + + // Handles Mqtt message + override def messageArrived(arg0: String, arg1: MqttMessage) { + blockGenerator += new String(arg1.getPayload()) + } + + override def deliveryComplete(arg0: IMqttDeliveryToken) { + } + + override def connectionLost(arg0: Throwable) { + logInfo("Connection lost " + arg0) + } + } + + // Set up callback for MqttClient + client.setCallback(callback) + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala new file mode 100644 index 0000000000..28a944f57e --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala @@ -0,0 +1,24 @@ +/* + * 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 + +package object mqtt { + implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc) +} + + diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index c1f41640dc..3ddb4d084f 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -19,6 +19,7 @@ package org.apache.spark.streaming.mqtt; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -28,14 +29,11 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { public void testMQTTStream() { String brokerUrl = "abc"; String topic = "def"; - JavaStreamingContextWithMQTT sscWithMQTT = new JavaStreamingContextWithMQTT(ssc); + MQTTFunctions mqttFunc = new MQTTFunctions(ssc); // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithMQTT.mqttStream(brokerUrl, topic); - JavaDStream test2 = sscWithMQTT.mqttStream(brokerUrl, topic, + JavaDStream test1 = mqttFunc.mqttStream(brokerUrl, topic); + JavaDStream test2 = mqttFunc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithMQTT.socketTextStream("localhost", 9999); } } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala new file mode 100644 index 0000000000..22e297a03a --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala @@ -0,0 +1,99 @@ +/* + * 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.api.java.twitter + +import twitter4j.Status +import twitter4j.auth.Authorization + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.twitter._ + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating Twitter input streams. + */ +class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + */ + def twitterStream(): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(None) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream(filters: Array[String]): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(None, filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(None, filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization + */ + def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(Some(twitterAuth)) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + twitterAuth: Authorization, + filters: Array[String] + ): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J Authorization object + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Authorization, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala deleted file mode 100644 index 0250364331..0000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala +++ /dev/null @@ -1,99 +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.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Twitter input streams. - */ -class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - */ - def twitterStream(): JavaDStream[Status] = { - ssc.twitterStream(None) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream(filters: Array[String]): JavaDStream[Status] = { - ssc.twitterStream(None, filters) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { - ssc.twitterStream(None, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - */ - def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth)) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String] - ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String], - storageLevel: StorageLevel - ): JavaDStream[Status] = { - ssc.twitterStream(Some(twitterAuth), filters, storageLevel) - } -} diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java index 34e4fbdd85..4564d6cd33 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java @@ -18,6 +18,8 @@ package org.apache.spark.streaming.twitter; import java.util.Arrays; + +import org.apache.spark.streaming.api.java.twitter.TwitterFunctions; import org.junit.Test; import twitter4j.Status; @@ -31,21 +33,18 @@ import org.apache.spark.streaming.api.java.JavaDStream; public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { @Test public void testTwitterStream() { - JavaStreamingContextWithTwitter sscWithTwitter = new JavaStreamingContextWithTwitter(ssc); + TwitterFunctions twitterFunc = new TwitterFunctions(ssc); String[] filters = (String[])Arrays.asList("filter1", "filter2").toArray(); Authorization auth = NullAuthorization.getInstance(); // tests the API, does not actually test data receiving - JavaDStream test1 = sscWithTwitter.twitterStream(); - JavaDStream test2 = sscWithTwitter.twitterStream(filters); + JavaDStream test1 = twitterFunc.twitterStream(); + JavaDStream test2 = twitterFunc.twitterStream(filters); JavaDStream test3 = - sscWithTwitter.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = sscWithTwitter.twitterStream(auth); - JavaDStream test5 = sscWithTwitter.twitterStream(auth, filters); + twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test4 = twitterFunc.twitterStream(auth); + JavaDStream test5 = twitterFunc.twitterStream(auth, filters); JavaDStream test6 = - sscWithTwitter.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithTwitter.socketTextStream("localhost", 9999); + twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala new file mode 100644 index 0000000000..a9bbce71f5 --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala @@ -0,0 +1,102 @@ +/* + * 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.api.java.zeromq + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import akka.actor.SupervisorStrategy +import akka.util.ByteString +import akka.zeromq.Subscribe + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.zeromq._ + +/** + * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra + * functions for creating ZeroMQ input streams. + */ +class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote ZeroMQ publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel Storage level to use for storing the received objects + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel, + supervisorStrategy: SupervisorStrategy + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + */ + def zeroMQStream[T]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn) + } +} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala deleted file mode 100644 index dc5d1f05be..0000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala +++ /dev/null @@ -1,102 +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.zeromq - -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - -import akka.actor.SupervisorStrategy -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating ZeroMQ input streams. - */ -class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext) - extends JavaStreamingContext(javaStreamingContext.ssc) { - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote ZeroMQ publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel Storage level to use for storing the received objects - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel, - supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - ssc.zeroMQStream[T](publisherUrl, subscribe, fn) - } -} diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index 96af7d737d..b020ae4cef 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming.zeromq; +import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions; import org.junit.Test; import akka.actor.SupervisorStrategy; @@ -32,7 +33,7 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @Test // tests the API, does not actually test data receiving public void testZeroMQStream() { - JavaStreamingContextWithZeroMQ sscWithZeroMQ = new JavaStreamingContextWithZeroMQ(ssc); + ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc); String publishUrl = "abc"; Subscribe subscribe = new Subscribe((ByteString)null); Function> bytesToObjects = new Function>() { @@ -42,14 +43,11 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { } }; - JavaDStream test1 = sscWithZeroMQ.zeroMQStream( + JavaDStream test1 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = sscWithZeroMQ.zeroMQStream( + JavaDStream test2 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = sscWithZeroMQ.zeroMQStream( + JavaDStream test3 = zeromqFunc.zeroMQStream( publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); - - // To verify that JavaStreamingContextWithKafka is also StreamingContext - JavaDStream socketStream = sscWithZeroMQ.socketTextStream("localhost", 9999); } } -- cgit v1.2.3 From a91f14cfdccd57cb80138d4905930c5b1b82ce1d Mon Sep 17 00:00:00 2001 From: prabeesh Date: Tue, 7 Jan 2014 12:21:20 +0530 Subject: spark -> org.apache.spark --- .../scala/org/apache/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../scala/org/apache/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../apache/spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../spark/streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../apache/spark/streaming/examples/clickstream/PageViewStream.scala | 4 ++-- 8 files changed, 12 insertions(+), 12 deletions(-) (limited to 'examples') 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 546495357f..4e0058cd70 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 @@ -134,9 +134,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 1486d77d8a..ea6ea67419 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 172091be2e..197461655e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index 74d76ec26c..6a32c75373 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index f43c8ab61d..002db57d59 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { 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 89d3042123..beda73a71b 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 @@ -62,9 +62,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index 1a40fdb9a3..4fe57de4a4 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -39,8 +39,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 0569846f18..807af199f4 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ object PageViewStream { def main(args: Array[String]) { -- cgit v1.2.3 From 4b92a20232bc24fd858ed4eb7c45462241e36829 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 7 Jan 2014 09:38:45 +0000 Subject: Issue #318 : minor style updates per review from Reynold Xin --- examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java | 5 +---- examples/src/main/java/org/apache/spark/examples/JavaKMeans.java | 3 --- examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java | 3 --- examples/src/main/java/org/apache/spark/examples/JavaPageRank.java | 4 ---- examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java | 3 --- examples/src/main/java/org/apache/spark/examples/JavaTC.java | 3 --- examples/src/main/java/org/apache/spark/examples/JavaWordCount.java | 3 --- examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java | 5 +---- .../src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java | 3 --- examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java | 3 --- 10 files changed, 2 insertions(+), 33 deletions(-) (limited to 'examples') diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 71bd3b4821..d552c47b22 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -35,9 +35,6 @@ public final class JavaHdfsLR { private static final int D = 10; // Number of dimensions private static final Random rand = new Random(42); - private JavaHdfsLR() { - } - static class DataPoint implements Serializable { DataPoint(double[] x, double y) { this.x = x; @@ -57,7 +54,7 @@ public final class JavaHdfsLR { double y = Double.parseDouble(tok[0]); double[] x = new double[D]; for (int i = 0; i < D; i++) { - x[i] = Double.parseDouble(tok[i+1]); + x[i] = Double.parseDouble(tok[i + 1]); } return new DataPoint(x, y); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 0808f33e6a..0dc879275a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -36,9 +36,6 @@ public final class JavaKMeans { private static final Pattern SPACE = Pattern.compile(" "); - private JavaKMeans() { - } - /** Parses numbers split by whitespace to a vector */ static Vector parseVector(String line) { String[] splits = SPACE.split(line); 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 d45d96d804..9eb1cadd71 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -54,9 +54,6 @@ public final class JavaLogQuery { public static final Pattern apacheLogRegex = Pattern.compile( "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); - private JavaLogQuery() { - } - /** Tracks the total query count and number of aggregate bytes for a particular group. */ public static class Stats implements Serializable { 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 12d2cce1a7..a84245b0c7 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,6 @@ package org.apache.spark.examples; -import org.apache.spark.SparkContext; import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -43,9 +42,6 @@ import java.util.regex.Pattern; public final class JavaPageRank { private static final Pattern SPACES = Pattern.compile("\\s+"); - private JavaPageRank() { - } - private static class Sum extends Function2 { @Override public Double call(Double a, Double b) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index f6ed510e05..3ec4a58d48 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -28,9 +28,6 @@ import java.util.List; /** Computes an approximation to pi */ public final class JavaSparkPi { - private JavaSparkPi() { - } - public static void main(String[] args) throws Exception { if (args.length == 0) { System.err.println("Usage: JavaLogQuery [slices]"); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 12b564d1ef..2ceb0fd94b 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -37,9 +37,6 @@ public final class JavaTC { private static final int numVertices = 100; private static final Random rand = new Random(42); - private JavaTC() { - } - static List> generateGraph() { Set> edges = new HashSet>(numEdges); while (edges.size() < numEdges) { 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 fc9beb8fe5..6651f98d56 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -32,9 +32,6 @@ import java.util.regex.Pattern; public final class JavaWordCount { private static final Pattern SPACE = Pattern.compile(" "); - private JavaWordCount() { - } - public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaWordCount "); 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 c42d9cb788..435a86e62a 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 @@ -33,10 +33,7 @@ import scala.Tuple2; /** * Example using MLLib ALS from Java. */ -public final class JavaALS { - - private JavaALS() { - } +public final class JavaALS { static class ParseRating extends Function { private static final Pattern COMMA = Pattern.compile(","); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 9d10473aed..4b2658f257 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -32,9 +32,6 @@ import java.util.regex.Pattern; */ public final class JavaKMeans { - private JavaKMeans() { - } - static class ParsePoint extends Function { private static final Pattern SPACE = Pattern.compile(" "); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index b057f71e08..21586ce817 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -34,9 +34,6 @@ import java.util.regex.Pattern; */ public final class JavaLR { - private JavaLR() { - } - static class ParsePoint extends Function { private static final Pattern COMMA = Pattern.compile(","); private static final Pattern SPACE = Pattern.compile(" "); -- cgit v1.2.3 From aa99f226a691ddcb4442d60f4cd4908f434cc4ce Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 7 Jan 2014 01:56:15 -0800 Subject: Removed XYZFunctions and added XYZUtils as a common Scala and Java interface for creating XYZ streams. --- .../streaming/examples/JavaFlumeEventCount.java | 5 +- .../streaming/examples/JavaKafkaWordCount.java | 9 +- .../spark/streaming/examples/FlumeEventCount.scala | 2 +- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/TwitterAlgebirdCMS.scala | 4 +- .../streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../streaming/examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 3 +- .../streaming/api/java/flume/FlumeFunctions.scala | 48 ------- .../spark/streaming/flume/FlumeFunctions.scala | 43 ------ .../apache/spark/streaming/flume/FlumeUtils.scala | 70 ++++++++++ .../org/apache/spark/streaming/flume/package.scala | 23 ---- .../streaming/flume/JavaFlumeStreamSuite.java | 13 +- .../spark/streaming/flume/FlumeStreamSuite.scala | 2 +- .../streaming/api/java/kafka/KafkaFunctions.scala | 107 -------------- .../spark/streaming/kafka/KafkaFunctions.scala | 73 ---------- .../apache/spark/streaming/kafka/KafkaUtils.scala | 153 +++++++++++++++++++++ .../org/apache/spark/streaming/kafka/package.scala | 23 ---- .../streaming/kafka/JavaKafkaStreamSuite.java | 11 +- .../spark/streaming/kafka/KafkaStreamSuite.scala | 8 +- .../streaming/api/java/mqtt/MQTTFunctions.scala | 59 -------- .../spark/streaming/mqtt/MQTTFunctions.scala | 43 ------ .../apache/spark/streaming/mqtt/MQTTUtils.scala | 75 ++++++++++ .../org/apache/spark/streaming/mqtt/package.scala | 24 ---- .../spark/streaming/mqtt/JavaMQTTStreamSuite.java | 6 +- .../spark/streaming/mqtt/MQTTStreamSuite.scala | 4 +- .../api/java/twitter/TwitterFunctions.scala | 99 ------------- .../spark/streaming/twitter/TwitterFunctions.scala | 49 ------- .../spark/streaming/twitter/TwitterUtils.scala | 126 +++++++++++++++++ .../apache/spark/streaming/twitter/package.scala | 22 --- .../streaming/twitter/JavaTwitterStreamSuite.java | 20 ++- .../streaming/twitter/TwitterStreamSuite.scala | 13 +- .../api/java/zeromq/ZeroMQFunctions.scala | 102 -------------- .../spark/streaming/zeromq/ZeroMQFunctions.scala | 57 -------- .../spark/streaming/zeromq/ZeroMQUtils.scala | 126 +++++++++++++++++ .../apache/spark/streaming/zeromq/package.scala | 24 ---- .../streaming/zeromq/JavaZeroMQStreamSuite.java | 17 +-- .../spark/streaming/zeromq/ZeroMQStreamSuite.scala | 8 +- 39 files changed, 608 insertions(+), 871 deletions(-) delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala delete mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala delete mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala delete mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala delete mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala (limited to 'examples') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 83900a18df..0a2b3def18 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -20,7 +20,7 @@ package org.apache.spark.streaming.examples; import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.api.java.flume.FlumeFunctions; +import org.apache.spark.streaming.flume.FlumeUtils; import org.apache.spark.streaming.flume.SparkFlumeEvent; /** @@ -53,8 +53,7 @@ public class JavaFlumeEventCount { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); - FlumeFunctions flumeFunc = new FlumeFunctions(ssc); - JavaDStream flumeStream = flumeFunc.flumeStream("localhost", port); + JavaDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 51de4054cc..3bd7a3a90e 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,7 +29,7 @@ import org.apache.spark.streaming.Duration; 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.api.java.kafka.KafkaFunctions; +import org.apache.spark.streaming.kafka.KafkaUtils; import scala.Tuple2; /** @@ -54,7 +54,7 @@ public class JavaKafkaWordCount { } // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", + JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount", new Duration(2000), System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); @@ -65,8 +65,7 @@ public class JavaKafkaWordCount { topicMap.put(topic, numThreads); } - KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); - JavaPairDStream messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap); + JavaPairDStream messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override @@ -96,6 +95,6 @@ public class JavaKafkaWordCount { }); wordCounts.print(); - ssc.start(); + jssc.start(); } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 149640e0d1..ae3709b3d9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -52,7 +52,7 @@ object FlumeEventCount { System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a flume stream - val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) + val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) // Print out the count of events received from this server in each batch stream.count().map(cnt => "Received " + cnt + " flume events." ).print() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 633712e816..022c8c5cb9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -53,7 +53,7 @@ object KafkaWordCount { ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2) + val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1l)) .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index f65c3f8b91..325290b66f 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -97,7 +97,7 @@ object MQTTWordCount { val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY) + val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index a60570f884..3ccdc908e2 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.twitter._ *

*

* - * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure + * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a data structure * for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc), * that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the * estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold @@ -63,7 +63,7 @@ object TwitterAlgebirdCMS { val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 1382fa4d1d..c7e83e76b0 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -50,7 +50,7 @@ object TwitterAlgebirdHLL { val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 84842b3d65..e2b0418d55 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -40,7 +40,7 @@ object TwitterPopularTags { val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - val stream = ssc.twitterStream(None, filters) + val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) 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 789c5f2d08..5a7673756e 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 @@ -85,11 +85,10 @@ object ZeroMQWordCount { 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) + val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() } - } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala deleted file mode 100644 index 3347d19796..0000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala +++ /dev/null @@ -1,48 +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.api.java.flume - -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.flume._ -import org.apache.spark.storage.StorageLevel - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Flume input streams. - */ -class FlumeFunctions(javaStreamingContext: JavaStreamingContext) { - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { - javaStreamingContext.ssc.flumeStream(hostname, port) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel): - JavaDStream[SparkFlumeEvent] = { - javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel) - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala deleted file mode 100644 index 35e7a01abc..0000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala +++ /dev/null @@ -1,43 +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.flume - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra Flume input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversion. Import org.apache.spark.streaming.flume._ to use these functions. - */ -class FlumeFunctions(ssc: StreamingContext) { - /** - * Create a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def flumeStream ( - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala new file mode 100644 index 0000000000..834b775d4f --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -0,0 +1,70 @@ +/* + * 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.flume + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} + +object FlumeUtils { + /** + * Create a input stream from a Flume source. + * @param ssc StreamingContext object + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream ( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[SparkFlumeEvent] = { + val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port, storageLevel) + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala deleted file mode 100644 index c087a39d1c..0000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala +++ /dev/null @@ -1,23 +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 - -package object flume { - implicit def sscToFlumeFunctions(ssc: StreamingContext) = new FlumeFunctions(ssc) -} - diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 5930fee925..733389b98d 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -1,4 +1,4 @@ -package org.apache.spark.streaming.flume;/* +/* * 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. @@ -15,21 +15,20 @@ package org.apache.spark.streaming.flume;/* * limitations under the License. */ +package org.apache.spark.streaming.flume; + import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.flume.FlumeFunctions; -import org.apache.spark.streaming.flume.SparkFlumeEvent; + import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { - FlumeFunctions flumeFunc = new FlumeFunctions(ssc); - // tests the API, does not actually test data receiving - JavaDStream test1 = flumeFunc.flumeStream("localhost", 12345); - JavaDStream test2 = flumeFunc.flumeStream("localhost", 12345, + JavaDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); + JavaDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 74840f6499..2e8e9fac45 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -39,7 +39,7 @@ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala deleted file mode 100644 index 491331bb37..0000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala +++ /dev/null @@ -1,107 +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.api.java.kafka - -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - -import java.lang.{Integer => JInt} -import java.util.{Map => JMap} - -import kafka.serializer.Decoder - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.kafka._ - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Kafka input streams. - */ -class KafkaFunctions(javaStreamingContext: JavaStreamingContext) { - - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages form 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 - * in its own thread. - * @param storageLevel RDD storage level. - * - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. Defaults to memory-only - */ - def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] - - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] - - javaStreamingContext.ssc.kafkaStream[K, V, U, T]( - kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala deleted file mode 100644 index 2135634a69..0000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala +++ /dev/null @@ -1,73 +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.kafka - -import scala.reflect.ClassTag - -import kafka.serializer.{Decoder, StringDecoder} - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra Kafka input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversion. Import org.apache.spark.streaming.kafka._ to use these functions. - */ -class KafkaFunctions(ssc: StreamingContext) { - /** - * Create an 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 - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def kafkaStream( - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - kafkaStream[String, String, StringDecoder, StringDecoder]( - kafkaParams, - topics, - storageLevel) - } - - /** - * Create an input stream that pulls messages from a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def kafkaStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): DStream[(K, V)] = { - val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 0000000000..c2d851f943 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,153 @@ +/* + * 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.kafka + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import java.lang.{Integer => JInt} +import java.util.{Map => JMap} + +import kafka.serializer.{Decoder, StringDecoder} + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} + + +object KafkaUtils { + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param zkQuorum Zookeeper 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 + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ + def createStream( + ssc: StreamingContext, + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, storageLevel) + } + + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): DStream[(K, V)] = { + val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper 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 + * in its own thread + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper 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 + * in its own thread. + * @param storageLevel RDD storage level. + * + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[String, String] = { + implicit val cmt: ClassTag[String] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of RDD + * @param valueTypeClass value type of RDD + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. Defaults to MEMORY_AND_DISK_2. + */ + def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val valueCmt: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + + implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] + implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + + createStream[K, V, U, T]( + jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala deleted file mode 100644 index 44e7ce6e1b..0000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala +++ /dev/null @@ -1,23 +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 - -package object kafka { - implicit def sscToKafkaFunctions(ssc: StreamingContext) = new KafkaFunctions(ssc) -} - diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index fdea96e506..7b4999447e 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,32 +18,27 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; - -import org.apache.spark.streaming.api.java.kafka.KafkaFunctions; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test public void testKafkaStream() { - HashMap topics = Maps.newHashMap(); - KafkaFunctions kafkaFunc = new KafkaFunctions(ssc); // tests the API, does not actually test data receiving - JavaPairDStream test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics); - JavaPairDStream test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics, + JavaPairDStream test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); + JavaPairDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = kafkaFunc.kafkaStream( + JavaPairDStream test3 = KafkaUtils.createStream(ssc, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 2ef3e99c55..9c81f23c19 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -28,11 +28,11 @@ class KafkaStreamSuite extends TestSuiteBase { val topics = Map("my-topic" -> 1) // tests the API, does not actually test data receiving - val test1 = ssc.kafkaStream("localhost:12345", "group", topics) - val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) + val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = ssc.kafkaStream[String, String, StringDecoder, StringDecoder]( - kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala deleted file mode 100644 index 72124956fc..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.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.api.java.mqtt - -import scala.reflect.ClassTag - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.mqtt._ - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating MQTT input streams. - */ -class MQTTFunctions(javaStreamingContext: JavaStreamingContext) { - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - */ - def mqttStream( - brokerUrl: String, - topic: String - ): JavaDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.mqttStream(brokerUrl, topic) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): JavaDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel) - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala deleted file mode 100644 index 86f4e9c724..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala +++ /dev/null @@ -1,43 +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.mqtt - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions. - */ -class MQTTFunctions(ssc: StreamingContext) { - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param brokerUrl Url of remote MQTT publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def mqttStream( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala new file mode 100644 index 0000000000..0e6c25dbee --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -0,0 +1,75 @@ +/* + * 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.mqtt + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} +import scala.reflect.ClassTag + +object MQTTUtils { + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param ssc StreamingContext object + * @param brokerUrl Url of remote MQTT publisher + * @param topic Topic name to subscribe to + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def createStream( + ssc: StreamingContext, + brokerUrl: String, + topic: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[String] = { + val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param jssc JavaStreamingContext object + * @param brokerUrl Url of remote MQTT publisher + * @param topic Topic name to subscribe to + */ + def createStream( + jssc: JavaStreamingContext, + brokerUrl: String, + topic: String + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, brokerUrl, topic) + } + + /** + * Create an input stream that receives messages pushed by a MQTT publisher. + * @param jssc JavaStreamingContext object + * @param brokerUrl Url of remote MQTT publisher + * @param topic Topic name to subscribe to + * @param storageLevel RDD storage level. + */ + def createStream( + jssc: JavaStreamingContext, + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ): JavaDStream[String] = { + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] + createStream(jssc.ssc, brokerUrl, topic, storageLevel) + } +} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala deleted file mode 100644 index 28a944f57e..0000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala +++ /dev/null @@ -1,24 +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 - -package object mqtt { - implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc) -} - - diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index 3ddb4d084f..44743aaecf 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -19,7 +19,6 @@ package org.apache.spark.streaming.mqtt; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -29,11 +28,10 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { public void testMQTTStream() { String brokerUrl = "abc"; String topic = "def"; - MQTTFunctions mqttFunc = new MQTTFunctions(ssc); // tests the API, does not actually test data receiving - JavaDStream test1 = mqttFunc.mqttStream(brokerUrl, topic); - JavaDStream test2 = mqttFunc.mqttStream(brokerUrl, topic, + JavaDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); + JavaDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index ab6542918b..fcc159e85a 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -28,8 +28,8 @@ class MQTTStreamSuite extends TestSuiteBase { val topic = "def" // tests the API, does not actually test data receiving - val test1 = ssc.mqttStream(brokerUrl, topic) - val test2 = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic) + val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala deleted file mode 100644 index 22e297a03a..0000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala +++ /dev/null @@ -1,99 +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.api.java.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.twitter._ - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating Twitter input streams. - */ -class TwitterFunctions(javaStreamingContext: JavaStreamingContext) { - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - */ - def twitterStream(): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream(filters: Array[String]): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None, filters) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(None, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - */ - def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth)) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String] - ): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Authorization, - filters: Array[String], - storageLevel: StorageLevel - ): JavaDStream[Status] = { - javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel) - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala deleted file mode 100644 index e91049d9b1..0000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala +++ /dev/null @@ -1,49 +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.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ - -/** - * Extra Twitter input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.twitter._ to use these functions. - */ -class TwitterFunctions(ssc: StreamingContext) { - /** - * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth - * authorization; this uses the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - twitterAuth: Option[Authorization], - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { - val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) - ssc.registerInputStream(inputStream) - inputStream - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala new file mode 100644 index 0000000000..5e506ffabc --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala @@ -0,0 +1,126 @@ +/* + * 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.twitter + +import twitter4j.Status +import twitter4j.auth.Authorization +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} + +object TwitterUtils { + /** + * Create a input stream that returns tweets received from Twitter. + * @param ssc StreamingContext object + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + ssc: StreamingContext, + twitterAuth: Option[Authorization], + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = { + val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) + ssc.registerInputStream(inputStream) + inputStream + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param jssc JavaStreamingContext object + */ + def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = { + createStream(jssc.ssc, None) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param jssc JavaStreamingContext object + * @param filters Set of filter strings to get only those tweets that match them + */ + def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = { + createStream(jssc.ssc, None, filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and + * twitter4j.oauth.accessTokenSecret. + * @param jssc JavaStreamingContext object + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + jssc: JavaStreamingContext, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + createStream(jssc.ssc, None, filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param jssc JavaStreamingContext object + * @param twitterAuth Twitter4J Authorization + */ + def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = { + createStream(jssc.ssc, Some(twitterAuth)) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param jssc JavaStreamingContext object + * @param twitterAuth Twitter4J Authorization + * @param filters Set of filter strings to get only those tweets that match them + */ + def createStream( + jssc: JavaStreamingContext, + twitterAuth: Authorization, + filters: Array[String] + ): JavaDStream[Status] = { + createStream(jssc.ssc, Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param jssc JavaStreamingContext object + * @param twitterAuth Twitter4J Authorization object + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + jssc: JavaStreamingContext, + twitterAuth: Authorization, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) + } +} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala deleted file mode 100644 index 23f82c5885..0000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala +++ /dev/null @@ -1,22 +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 - -package object twitter { - implicit def sscToTwitterFunctions(ssc: StreamingContext) = new TwitterFunctions(ssc) -} diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java index 4564d6cd33..e46b4e5c75 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java @@ -19,13 +19,10 @@ package org.apache.spark.streaming.twitter; import java.util.Arrays; -import org.apache.spark.streaming.api.java.twitter.TwitterFunctions; import org.junit.Test; - import twitter4j.Status; import twitter4j.auth.Authorization; import twitter4j.auth.NullAuthorization; - import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; @@ -33,18 +30,17 @@ import org.apache.spark.streaming.api.java.JavaDStream; public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { @Test public void testTwitterStream() { - TwitterFunctions twitterFunc = new TwitterFunctions(ssc); String[] filters = (String[])Arrays.asList("filter1", "filter2").toArray(); Authorization auth = NullAuthorization.getInstance(); // tests the API, does not actually test data receiving - JavaDStream test1 = twitterFunc.twitterStream(); - JavaDStream test2 = twitterFunc.twitterStream(filters); - JavaDStream test3 = - twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = twitterFunc.twitterStream(auth); - JavaDStream test5 = twitterFunc.twitterStream(auth, filters); - JavaDStream test6 = - twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test1 = TwitterUtils.createStream(ssc); + JavaDStream test2 = TwitterUtils.createStream(ssc, filters); + JavaDStream test3 = TwitterUtils.createStream( + ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test4 = TwitterUtils.createStream(ssc, auth); + JavaDStream test5 = TwitterUtils.createStream(ssc, auth, filters); + JavaDStream test6 = TwitterUtils.createStream(ssc, + auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala index d7f6d35e07..a0a8fe617b 100644 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -29,12 +29,13 @@ class TwitterStreamSuite extends TestSuiteBase { val authorization: Authorization = NullAuthorization.getInstance() // tests the API, does not actually test data receiving - val test1 = ssc.twitterStream(None) - val test2 = ssc.twitterStream(None, filters) - val test3 = ssc.twitterStream(None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4 = ssc.twitterStream(Some(authorization)) - val test5 = ssc.twitterStream(Some(authorization), filters) - val test6 = ssc.twitterStream(Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1 = TwitterUtils.createStream(ssc, None) + val test2 = TwitterUtils.createStream(ssc, None, filters) + val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4 = TwitterUtils.createStream(ssc, Some(authorization)) + val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters) + val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters, + StorageLevel.MEMORY_AND_DISK_SER_2) // Note that actually testing the data receiving is hard as authentication keys are // necessary for accessing Twitter live stream diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala deleted file mode 100644 index a9bbce71f5..0000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala +++ /dev/null @@ -1,102 +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.api.java.zeromq - -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - -import akka.actor.SupervisorStrategy -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.zeromq._ - -/** - * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra - * functions for creating ZeroMQ input streams. - */ -class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) { - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote ZeroMQ publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel Storage level to use for storing the received objects - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel, - supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - */ - def zeroMQStream[T]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator - javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn) - } -} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala deleted file mode 100644 index f4c75ab7c9..0000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala +++ /dev/null @@ -1,57 +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.zeromq - -import scala.reflect.ClassTag - -import akka.actor.{Props, SupervisorStrategy} -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ -import org.apache.spark.streaming.receivers._ - -/** - * Extra ZeroMQ input stream functions available on [[org.apache.spark.streaming.StreamingContext]] - * through implicit conversions. Import org.apache.spark.streaming.zeromq._ to use these functions. - */ -class ZeroMQFunctions(ssc: StreamingContext) { - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic - * and each frame has sequence of byte thus it needs the converter - * (which might be deserializer of bytes) to translate from sequence - * of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to memory-only. - */ - def zeroMQStream[T: ClassTag]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] ⇒ Iterator[T], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { - ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), - "ZeroMQReceiver", storageLevel, supervisorStrategy) - } -} - diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala new file mode 100644 index 0000000000..546d9df3b5 --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -0,0 +1,126 @@ +/* + * 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.zeromq + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ +import akka.actor.{Props, SupervisorStrategy} +import akka.util.ByteString +import akka.zeromq.Subscribe +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy +import org.apache.spark.streaming.{StreamingContext, DStream} +import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} + +object ZeroMQUtils { + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param ssc StreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic + * and each frame has sequence of byte thus it needs the converter + * (which might be deserializer of bytes) to translate from sequence + * of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. + */ + def createStream[T: ClassTag]( + ssc: StreamingContext, + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] ⇒ Iterator[T], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { + ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), + "ZeroMQReceiver", storageLevel, supervisorStrategy) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote ZeroMQ publisher + * @param subscribe Topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream[T]( + jssc: JavaStreamingContext, + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel, + supervisorStrategy: SupervisorStrategy + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. + */ + def createStream[T]( + jssc: JavaStreamingContext, + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param jssc JavaStreamingContext object + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe Topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + */ + def createStream[T]( + jssc: JavaStreamingContext, + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] + ): JavaDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + createStream[T](jssc.ssc, publisherUrl, subscribe, fn) + } +} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala deleted file mode 100644 index dc27178149..0000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala +++ /dev/null @@ -1,24 +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 - -package object zeromq { - implicit def sscToZeroMQFunctions(ssc: StreamingContext) = new ZeroMQFunctions(ssc) -} - - diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index b020ae4cef..d2361e14b8 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,13 +17,10 @@ package org.apache.spark.streaming.zeromq; -import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions; import org.junit.Test; - import akka.actor.SupervisorStrategy; import akka.util.ByteString; import akka.zeromq.Subscribe; - import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -33,7 +30,6 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @Test // tests the API, does not actually test data receiving public void testZeroMQStream() { - ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc); String publishUrl = "abc"; Subscribe subscribe = new Subscribe((ByteString)null); Function> bytesToObjects = new Function>() { @@ -43,11 +39,12 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { } }; - JavaDStream test1 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = zeromqFunc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); + JavaDStream test1 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects); + JavaDStream test2 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaDStream test3 = ZeroMQUtils.createStream( + ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), + SupervisorStrategy.defaultStrategy()); } } diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala index 5adcdb821f..4193b8a02f 100644 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -33,10 +33,10 @@ class ZeroMQStreamSuite extends TestSuiteBase { val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] // tests the API, does not actually test data receiving - val test1 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects) - val test2 = ssc.zeroMQStream( - publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects, + val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) + val test2 = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) + val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) // TODO: Actually test data receiving -- cgit v1.2.3 From 8f02f1c3d45ee553ed6bec2dc81fbae4435274fc Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 7 Jan 2014 11:02:29 -0800 Subject: Fixed examples/pom.xml and run-example based on Patrick's suggestions. --- bin/run-example | 7 +------ examples/pom.xml | 7 +------ 2 files changed, 2 insertions(+), 12 deletions(-) (limited to 'examples') diff --git a/bin/run-example b/bin/run-example index dfb4bf7baf..a94913ddef 100755 --- a/bin/run-example +++ b/bin/run-example @@ -45,20 +45,15 @@ fi EXAMPLES_DIR="$FWDIR"/examples SPARK_EXAMPLES_JAR="" if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` fi -if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar` -fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 echo "You need to build Spark with sbt assembly before running this program" >&2 exit 1 fi + # Since the examples JAR ideally shouldn't include spark-core (that dependency should be # "provided"), also add our standard Spark classpath, built using compute-classpath.sh. CLASSPATH=`$FWDIR/bin/compute-classpath.sh` diff --git a/examples/pom.xml b/examples/pom.xml index 1839667fea..9c77640100 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -75,31 +75,26 @@ org.apache.spark spark-streaming-twitter_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-kafka_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-flume_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-zeromq_${scala.binary.version} ${project.version} - provided org.apache.spark spark-streaming-mqtt_${scala.binary.version} ${project.version} - provided org.apache.hbase @@ -189,7 +184,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar *:* -- cgit v1.2.3 From e688e11206401850a13a87d7db52941cc716f88a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 12:42:19 -0800 Subject: Add log4j exclusion rule to maven. To make this work I had to rename the defaults file. Otherwise maven's pattern matching rules included it when trying to match other log4j.properties files. I also fixed a bug in the existing maven build where two tags were present in assembly/pom.xml such that one overwrote the other. --- assembly/pom.xml | 6 +++--- .../main/resources/org/apache/spark/default-log4j.properties | 8 -------- .../main/resources/org/apache/spark/log4j-defaults.properties | 8 ++++++++ core/src/main/scala/org/apache/spark/Logging.scala | 10 ++++++---- examples/pom.xml | 3 +++ 5 files changed, 20 insertions(+), 15 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/default-log4j.properties create mode 100644 core/src/main/resources/org/apache/spark/log4j-defaults.properties (limited to 'examples') diff --git a/assembly/pom.xml b/assembly/pom.xml index 9b70812c64..54a25910ce 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -108,12 +108,12 @@ META-INF/services/org.apache.hadoop.fs.FileSystem - - - reference.conf + + log4j.properties + diff --git a/core/src/main/resources/org/apache/spark/default-log4j.properties b/core/src/main/resources/org/apache/spark/default-log4j.properties deleted file mode 100644 index d72dbadc39..0000000000 --- a/core/src/main/resources/org/apache/spark/default-log4j.properties +++ /dev/null @@ -1,8 +0,0 @@ -# Set everything to be logged to the console -log4j.rootCategory=INFO, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties new file mode 100644 index 0000000000..d72dbadc39 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -0,0 +1,8 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index d519fc5a29..4a34989e50 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -104,13 +104,15 @@ trait Logging { // If Log4j doesn't seem initialized, load a default properties file val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { - val defaultLogProps = "org/apache/spark/default-log4j.properties" + val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => PropertyConfigurator.configure(url) - case None => System.err.println(s"Spark was unable to load $defaultLogProps") + case Some(url) => + PropertyConfigurator.configure(url) + log.info(s"Using Spark's default log4j profile: $defaultLogProps") + case None => + System.err.println(s"Spark was unable to load $defaultLogProps") } - log.info(s"Using Spark's default log4j profile: $defaultLogProps") } Logging.initialized = true diff --git a/examples/pom.xml b/examples/pom.xml index 7a7032c319..7e41bef252 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -203,6 +203,9 @@ reference.conf + + log4j.properties + -- cgit v1.2.3