Merge branch 'master' into stageInfo

This commit is contained in:
Imran Rashid 2013-03-03 15:20:46 -08:00
commit d36abdb053
136 changed files with 4163 additions and 1582 deletions

2
.gitignore vendored
View file

@ -34,3 +34,5 @@ log/
spark-tests.log spark-tests.log
streaming-tests.log streaming-tests.log
dependency-reduced-pom.xml dependency-reduced-pom.xml
.ensime
.ensime_lucene

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>

View file

@ -23,6 +23,7 @@ import spark.partial.BoundedDouble
import spark.partial.PartialResult import spark.partial.PartialResult
import spark.rdd._ import spark.rdd._
import spark.SparkContext._ import spark.SparkContext._
import spark.Partitioner._
/** /**
* Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
@ -248,8 +249,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
} }
/** /**
* Simplified version of combineByKey that hash-partitions the resulting RDD using the default * Simplified version of combineByKey that hash-partitions the resulting RDD using the
* parallelism level. * existing partitioner/parallelism level.
*/ */
def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C)
: RDD[(K, C)] = { : RDD[(K, C)] = {
@ -259,7 +260,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/** /**
* Merge the values for each key using an associative reduce function. This will also perform * Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a * the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level. * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/
* parallelism level.
*/ */
def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { def reduceByKey(func: (V, V) => V): RDD[(K, V)] = {
reduceByKey(defaultPartitioner(self), func) reduceByKey(defaultPartitioner(self), func)
@ -267,7 +269,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/** /**
* Group the values for each key in the RDD into a single sequence. Hash-partitions the * Group the values for each key in the RDD into a single sequence. Hash-partitions the
* resulting RDD with the default parallelism level. * resulting RDD with the existing partitioner/parallelism level.
*/ */
def groupByKey(): RDD[(K, Seq[V])] = { def groupByKey(): RDD[(K, Seq[V])] = {
groupByKey(defaultPartitioner(self)) groupByKey(defaultPartitioner(self))
@ -295,7 +297,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
* resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
* pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
* using the default level of parallelism. * using the existing partitioner/parallelism level.
*/ */
def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = {
leftOuterJoin(other, defaultPartitioner(self, other)) leftOuterJoin(other, defaultPartitioner(self, other))
@ -315,7 +317,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
* resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
* pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
* RDD using the default parallelism level. * RDD using the existing partitioner/parallelism level.
*/ */
def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = {
rightOuterJoin(other, defaultPartitioner(self, other)) rightOuterJoin(other, defaultPartitioner(self, other))
@ -438,17 +440,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
cogroup(other1, other2, defaultPartitioner(self, other1, other2)) cogroup(other1, other2, defaultPartitioner(self, other1, other2))
} }
/**
* Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of
* the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner.
*/
def defaultPartitioner(rdds: RDD[_]*): Partitioner = {
for (r <- rdds if r.partitioner != None) {
return r.partitioner.get
}
return new HashPartitioner(self.context.defaultParallelism)
}
/** /**
* Return the list of values in the RDD for key `key`. This operation is done efficiently if the * Return the list of values in the RDD for key `key`. This operation is done efficiently if the
* RDD has a known partitioner by only searching the partition that the key maps to. * RDD has a known partitioner by only searching the partition that the key maps to.

View file

@ -9,6 +9,35 @@ abstract class Partitioner extends Serializable {
def getPartition(key: Any): Int def getPartition(key: Any): Int
} }
object Partitioner {
/**
* Choose a partitioner to use for a cogroup-like operation between a number of RDDs.
*
* If any of the RDDs already has a partitioner, choose that one.
*
* Otherwise, we use a default HashPartitioner. For the number of partitions, if
* spark.default.parallelism is set, then we'll use the value from SparkContext
* defaultParallelism, otherwise we'll use the max number of upstream partitions.
*
* Unless spark.default.parallelism is set, He number of partitions will be the
* same as the number of partitions in the largest upstream RDD, as this should
* be least likely to cause out-of-memory errors.
*
* We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD.
*/
def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
for (r <- bySize if r.partitioner != None) {
return r.partitioner.get
}
if (System.getProperty("spark.default.parallelism") != null) {
return new HashPartitioner(rdd.context.defaultParallelism)
} else {
return new HashPartitioner(bySize.head.partitions.size)
}
}
}
/** /**
* A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`. * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
* *

View file

@ -16,6 +16,7 @@ import org.apache.hadoop.mapred.TextOutputFormat
import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
import spark.Partitioner._
import spark.partial.BoundedDouble import spark.partial.BoundedDouble
import spark.partial.CountEvaluator import spark.partial.CountEvaluator
import spark.partial.GroupedCountEvaluator import spark.partial.GroupedCountEvaluator
@ -30,6 +31,7 @@ import spark.rdd.MapPartitionsRDD
import spark.rdd.MapPartitionsWithIndexRDD import spark.rdd.MapPartitionsWithIndexRDD
import spark.rdd.PipedRDD import spark.rdd.PipedRDD
import spark.rdd.SampledRDD import spark.rdd.SampledRDD
import spark.rdd.SubtractedRDD
import spark.rdd.UnionRDD import spark.rdd.UnionRDD
import spark.rdd.ZippedRDD import spark.rdd.ZippedRDD
import spark.storage.StorageLevel import spark.storage.StorageLevel
@ -299,19 +301,26 @@ abstract class RDD[T: ClassManifest](
*/ */
def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) def cartesian[U: ClassManifest](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])] =
groupBy[K](f, defaultPartitioner(this))
/** /**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
* mapping to that key. * mapping to that key.
*/ */
def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = { def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
val cleanF = sc.clean(f) groupBy(f, new HashPartitioner(numPartitions))
this.map(t => (cleanF(t), t)).groupByKey(numPartitions)
}
/** /**
* Return an RDD of grouped items. * Return an RDD of grouped items.
*/ */
def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = groupBy[K](f, sc.defaultParallelism) def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
val cleanF = sc.clean(f)
this.map(t => (cleanF(t), t)).groupByKey(p)
}
/** /**
* Return an RDD created by piping elements to a forked external process. * Return an RDD created by piping elements to a forked external process.
@ -393,6 +402,26 @@ abstract class RDD[T: ClassManifest](
filter(f.isDefinedAt).map(f) filter(f.isDefinedAt).map(f)
} }
/**
* 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.
*/
def subtract(other: RDD[T]): RDD[T] =
subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size)))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: RDD[T], numPartitions: Int): RDD[T] =
subtract(other, new HashPartitioner(numPartitions))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: RDD[T], p: Partitioner): RDD[T] = new SubtractedRDD[T](this, other, p)
/** /**
* Reduces the elements of this RDD using the specified commutative and associative binary operator. * Reduces the elements of this RDD using the specified commutative and associative binary operator.
*/ */

View file

@ -59,7 +59,7 @@ class SparkContext(
val appName: String, val appName: String,
val sparkHome: String = null, val sparkHome: String = null,
val jars: Seq[String] = Nil, val jars: Seq[String] = Nil,
environment: Map[String, String] = Map()) val environment: Map[String, String] = Map())
extends Logging { extends Logging {
// Ensure logging is initialized before we spawn any threads // Ensure logging is initialized before we spawn any threads
@ -434,7 +434,7 @@ class SparkContext(
} }
/** /**
* Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for * Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for
* reading it in distributed functions. The variable will be sent to each cluster only once. * reading it in distributed functions. The variable will be sent to each cluster only once.
*/ */
def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
@ -696,7 +696,7 @@ class SparkContext(
checkpointDir = Some(dir) checkpointDir = Some(dir)
} }
/** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */ /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
def defaultParallelism: Int = taskScheduler.defaultParallelism def defaultParallelism: Int = taskScheduler.defaultParallelism
/** Default min number of partitions for Hadoop RDDs when not given by user */ /** Default min number of partitions for Hadoop RDDs when not given by user */

View file

@ -6,8 +6,8 @@ import spark.api.java.function.{Function => JFunction}
import spark.util.StatCounter import spark.util.StatCounter
import spark.partial.{BoundedDouble, PartialResult} import spark.partial.{BoundedDouble, PartialResult}
import spark.storage.StorageLevel import spark.storage.StorageLevel
import java.lang.Double import java.lang.Double
import spark.Partitioner
class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
@ -57,6 +57,27 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
*/ */
def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions)) def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions))
/**
* 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.
*/
def subtract(other: JavaDoubleRDD): JavaDoubleRDD =
fromRDD(srdd.subtract(other))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaDoubleRDD, numPartitions: Int): JavaDoubleRDD =
fromRDD(srdd.subtract(other, numPartitions))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaDoubleRDD, p: Partitioner): JavaDoubleRDD =
fromRDD(srdd.subtract(other, p))
/** /**
* Return a sampled subset of this RDD. * Return a sampled subset of this RDD.
*/ */

View file

@ -19,6 +19,7 @@ import spark.OrderedRDDFunctions
import spark.storage.StorageLevel import spark.storage.StorageLevel
import spark.HashPartitioner import spark.HashPartitioner
import spark.Partitioner import spark.Partitioner
import spark.Partitioner._
import spark.RDD import spark.RDD
import spark.SparkContext.rddToPairRDDFunctions import spark.SparkContext.rddToPairRDDFunctions
@ -59,7 +60,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/** /**
* Return a new RDD containing only the elements that satisfy a predicate. * Return a new RDD containing only the elements that satisfy a predicate.
*/ */
def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue()))
/** /**
@ -102,7 +103,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* In addition, users can control the partitioning of the output RDD, and whether to perform * In addition, users can control the partitioning of the output RDD, and whether to perform
* map-side aggregation (if a mapper can produce multiple items with the same key). * map-side aggregation (if a mapper can produce multiple items with the same key).
*/ */
def combineByKey[C](createCombiner: Function[V, C], def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C], mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C], mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner): JavaPairRDD[K, C] = { partitioner: Partitioner): JavaPairRDD[K, C] = {
@ -181,6 +182,27 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] =
fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions)))
/**
* 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.
*/
def subtract(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] =
fromRDD(rdd.subtract(other))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaPairRDD[K, V], numPartitions: Int): JavaPairRDD[K, V] =
fromRDD(rdd.subtract(other, numPartitions))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaPairRDD[K, V], p: Partitioner): JavaPairRDD[K, V] =
fromRDD(rdd.subtract(other, p))
/** /**
* Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine` * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine`
* is true, Spark will group values of the same key together on the map side before the * is true, Spark will group values of the same key together on the map side before the
@ -220,30 +242,30 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
fromRDD(rdd.rightOuterJoin(other, partitioner)) fromRDD(rdd.rightOuterJoin(other, partitioner))
/** /**
* Simplified version of combineByKey that hash-partitions the resulting RDD using the default * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing
* parallelism level. * partitioner/parallelism level.
*/ */
def combineByKey[C](createCombiner: JFunction[V, C], def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C], mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = {
implicit val cm: ClassManifest[C] = implicit val cm: ClassManifest[C] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners)) fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd)))
} }
/** /**
* Merge the values for each key using an associative reduce function. This will also perform * Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a * the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level. * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/
* parallelism level.
*/ */
def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = { def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = {
val partitioner = rdd.defaultPartitioner(rdd) fromRDD(reduceByKey(defaultPartitioner(rdd), func))
fromRDD(reduceByKey(partitioner, func))
} }
/** /**
* Group the values for each key in the RDD into a single sequence. Hash-partitions the * Group the values for each key in the RDD into a single sequence. Hash-partitions the
* resulting RDD with the default parallelism level. * resulting RDD with the existing partitioner/parallelism level.
*/ */
def groupByKey(): JavaPairRDD[K, JList[V]] = def groupByKey(): JavaPairRDD[K, JList[V]] =
fromRDD(groupByResultToJava(rdd.groupByKey())) fromRDD(groupByResultToJava(rdd.groupByKey()))
@ -268,7 +290,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
* resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
* pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
* using the default level of parallelism. * using the existing partitioner/parallelism level.
*/ */
def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Option[W])] = def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Option[W])] =
fromRDD(rdd.leftOuterJoin(other)) fromRDD(rdd.leftOuterJoin(other))
@ -286,7 +308,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
* resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
* pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
* RDD using the default parallelism level. * RDD using the existing partitioner/parallelism level.
*/ */
def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Option[V], W)] = def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Option[V], W)] =
fromRDD(rdd.rightOuterJoin(other)) fromRDD(rdd.rightOuterJoin(other))
@ -309,7 +331,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* Pass each value in the key-value pair RDD through a map function without changing the keys; * Pass each value in the key-value pair RDD through a map function without changing the keys;
* this also retains the original RDD's partitioning. * this also retains the original RDD's partitioning.
*/ */
def mapValues[U](f: Function[V, U]): JavaPairRDD[K, U] = { def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = {
implicit val cm: ClassManifest[U] = implicit val cm: ClassManifest[U] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
fromRDD(rdd.mapValues(f)) fromRDD(rdd.mapValues(f))

View file

@ -55,6 +55,26 @@ JavaRDDLike[T, JavaRDD[T]] {
*/ */
def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd)) def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd))
/**
* 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.
*/
def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaRDD[T], numPartitions: Int): JavaRDD[T] =
wrapRDD(rdd.subtract(other, numPartitions))
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] =
wrapRDD(rdd.subtract(other, p))
} }
object JavaRDD { object JavaRDD {

View file

@ -12,7 +12,7 @@ import spark.storage.StorageLevel
import com.google.common.base.Optional import com.google.common.base.Optional
trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround[T] { trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def wrapRDD(rdd: RDD[T]): This def wrapRDD(rdd: RDD[T]): This
implicit val classManifest: ClassManifest[T] implicit val classManifest: ClassManifest[T]
@ -82,12 +82,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
} }
/** /**
* Part of the workaround for SPARK-668; called in PairFlatMapWorkaround.java. * Return a new RDD by first applying a function to all elements of this
* RDD, and then flattening the results.
*/ */
private[spark] def doFlatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = { def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala def fn = (x: T) => f.apply(x).asScala
def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType()) JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType())
} }
@ -110,8 +111,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
/** /**
* Return a new RDD by applying a function to each partition of this RDD. * Return a new RDD by applying a function to each partition of this RDD.
*/ */
def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V]): def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]):
JavaPairRDD[K, V] = { JavaPairRDD[K2, V2] = {
def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType()) JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType())
} }

View file

@ -31,8 +31,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @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 appName A name for your application, to display on the cluster web UI
* @param sparkHome The SPARK_HOME directory on the slave nodes * @param sparkHome The SPARK_HOME directory on the slave nodes
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file * @param jarFile JAR file to send to the cluster. This can be a path on the local file system
* system or HDFS, HTTP, HTTPS, or FTP URLs. * or an HDFS, HTTP, HTTPS, or FTP URL.
*/ */
def this(master: String, appName: String, sparkHome: String, jarFile: String) = def this(master: String, appName: String, sparkHome: String, jarFile: String) =
this(new SparkContext(master, appName, sparkHome, Seq(jarFile))) this(new SparkContext(master, appName, sparkHome, Seq(jarFile)))

View file

@ -1,20 +0,0 @@
package spark.api.java;
import spark.api.java.JavaPairRDD;
import spark.api.java.JavaRDDLike;
import spark.api.java.function.PairFlatMapFunction;
import java.io.Serializable;
/**
* Workaround for SPARK-668.
*/
class PairFlatMapWorkaround<T> implements Serializable {
/**
* Return a new RDD by first applying a function to all elements of this
* RDD, and then flattening the results.
*/
public <K, V> JavaPairRDD<K, V> flatMap(PairFlatMapFunction<T, K, V> f) {
return ((JavaRDDLike <T, ?>) this).doFlatMap(f);
}
}

View file

@ -51,7 +51,7 @@ private[spark] class PythonRDD[T: ClassManifest](
val env = SparkEnv.get val env = SparkEnv.get
// Start a thread to print the process's stderr to ours // Start a thread to print the process's stderr to ours
new Thread("stderr reader for " + command) { new Thread("stderr reader for " + pythonExec) {
override def run() { override def run() {
for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { for (line <- Source.fromInputStream(proc.getErrorStream).getLines) {
System.err.println(line) System.err.println(line)
@ -60,7 +60,7 @@ private[spark] class PythonRDD[T: ClassManifest](
}.start() }.start()
// Start a thread to feed the process input from our parent's iterator // Start a thread to feed the process input from our parent's iterator
new Thread("stdin writer for " + command) { new Thread("stdin writer for " + pythonExec) {
override def run() { override def run() {
SparkEnv.set(env) SparkEnv.set(env)
val out = new PrintWriter(proc.getOutputStream) val out = new PrintWriter(proc.getOutputStream)

View file

@ -33,6 +33,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
val waitingApps = new ArrayBuffer[ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo]
val completedApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo]
var firstApp: Option[ApplicationInfo] = None
val masterPublicAddress = { val masterPublicAddress = {
val envVar = System.getenv("SPARK_PUBLIC_DNS") val envVar = System.getenv("SPARK_PUBLIC_DNS")
if (envVar != null) envVar else ip if (envVar != null) envVar else ip
@ -41,7 +43,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
// As a temporary workaround before better ways of configuring memory, we allow users to set // 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 // 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. // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
override def preStart() { override def preStart() {
logInfo("Starting Spark master at spark://" + ip + ":" + port) logInfo("Starting Spark master at spark://" + ip + ":" + port)
@ -167,7 +169,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
// Try to spread out each app among all the nodes, until it has all its cores // Try to spread out each app among all the nodes, until it has all its cores
for (app <- waitingApps if app.coresLeft > 0) { for (app <- waitingApps if app.coresLeft > 0) {
val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE)
.filter(canUse(app, _)).sortBy(_.coresFree).reverse .filter(canUse(app, _)).sortBy(_.coresFree).reverse
val numUsable = usableWorkers.length val numUsable = usableWorkers.length
val assigned = new Array[Int](numUsable) // Number of cores to give on each node val assigned = new Array[Int](numUsable) // Number of cores to give on each node
var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)
@ -190,7 +192,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
} }
} else { } else {
// Pack each app into as few nodes as possible until we've assigned all its cores // Pack each app into as few nodes as possible until we've assigned all its cores
for (worker <- workers if worker.coresFree > 0) { for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
for (app <- waitingApps if app.coresLeft > 0) { for (app <- waitingApps if app.coresLeft > 0) {
if (canUse(app, worker)) { if (canUse(app, worker)) {
val coresToUse = math.min(worker.coresFree, app.coresLeft) val coresToUse = math.min(worker.coresFree, app.coresLeft)
@ -245,6 +247,13 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
idToApp(app.id) = app idToApp(app.id) = app
actorToApp(driver) = app actorToApp(driver) = app
addressToApp(driver.path.address) = app addressToApp(driver.path.address) = app
if (firstApp == None) {
firstApp = Some(app)
}
val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray
if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) {
logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
}
return app return app
} }
@ -254,7 +263,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
apps -= app apps -= app
idToApp -= app.id idToApp -= app.id
actorToApp -= app.driver actorToApp -= app.driver
addressToWorker -= app.driver.path.address addressToApp -= app.driver.path.address
completedApps += app // Remember it in our history completedApps += app // Remember it in our history
waitingApps -= app waitingApps -= app
for (exec <- app.executors.values) { for (exec <- app.executors.values) {

View file

@ -198,7 +198,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
outbox.synchronized { outbox.synchronized {
outbox.addMessage(message) outbox.addMessage(message)
if (channel.isConnected) { if (channel.isConnected) {
changeConnectionKeyInterest(SelectionKey.OP_WRITE) changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
} }
} }
} }
@ -219,7 +219,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
def finishConnect() { def finishConnect() {
try { try {
channel.finishConnect channel.finishConnect
changeConnectionKeyInterest(SelectionKey.OP_WRITE) changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending")
} catch { } catch {
case e: Exception => { case e: Exception => {
@ -239,8 +239,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
currentBuffers ++= chunk.buffers currentBuffers ++= chunk.buffers
} }
case None => { case None => {
changeConnectionKeyInterest(0) changeConnectionKeyInterest(SelectionKey.OP_READ)
/*key.interestOps(0)*/
return return
} }
} }
@ -267,6 +266,23 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
} }
} }
} }
override def read() {
// We don't expect the other side to send anything; so, we just read to detect an error or EOF.
try {
val length = channel.read(ByteBuffer.allocate(1))
if (length == -1) { // EOF
close()
} else if (length > 0) {
logWarning("Unexpected data read from SendingConnection to " + remoteConnectionManagerId)
}
} catch {
case e: Exception =>
logError("Exception while reading SendingConnection to " + remoteConnectionManagerId, e)
callOnExceptionCallback(e)
close()
}
}
} }

View file

@ -22,10 +22,10 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
override def getPartitions: Array[Partition] = { override def getPartitions: Array[Partition] = {
val dirContents = fs.listStatus(new Path(checkpointPath)) val dirContents = fs.listStatus(new Path(checkpointPath))
val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted
val numPartitions = splitFiles.size val numPartitions = partitionFiles.size
if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || if (numPartitions > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
!splitFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1))) { ! partitionFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1)))) {
throw new SparkException("Invalid checkpoint directory: " + checkpointPath) throw new SparkException("Invalid checkpoint directory: " + checkpointPath)
} }
Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i)) Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i))

View file

@ -15,7 +15,7 @@ import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.util.ReflectionUtils
import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
/** /**
@ -42,7 +42,7 @@ class HadoopRDD[K, V](
keyClass: Class[K], keyClass: Class[K],
valueClass: Class[V], valueClass: Class[V],
minSplits: Int) minSplits: Int)
extends RDD[(K, V)](sc, Nil) { extends RDD[(K, V)](sc, Nil) with Logging {
// A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@ -71,7 +71,7 @@ class HadoopRDD[K, V](
reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL)
// Register an on-task-completion callback to close the input stream. // Register an on-task-completion callback to close the input stream.
context.addOnCompleteCallback(() => reader.close()) context.addOnCompleteCallback{ () => close() }
val key: K = reader.createKey() val key: K = reader.createKey()
val value: V = reader.createValue() val value: V = reader.createValue()
@ -88,9 +88,6 @@ class HadoopRDD[K, V](
} }
gotNext = true gotNext = true
} }
if (finished) {
reader.close()
}
!finished !finished
} }
@ -104,6 +101,14 @@ class HadoopRDD[K, V](
gotNext = false gotNext = false
(key, value) (key, value)
} }
private def close() {
try {
reader.close()
} catch {
case e: Exception => logWarning("Exception in RecordReader.close()", e)
}
}
} }
override def getPreferredLocations(split: Partition): Seq[String] = { override def getPreferredLocations(split: Partition): Seq[String] = {

View file

@ -7,7 +7,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.Writable import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce._
import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
private[spark] private[spark]
@ -26,7 +26,8 @@ class NewHadoopRDD[K, V](
valueClass: Class[V], valueClass: Class[V],
@transient conf: Configuration) @transient conf: Configuration)
extends RDD[(K, V)](sc, Nil) extends RDD[(K, V)](sc, Nil)
with HadoopMapReduceUtil { with HadoopMapReduceUtil
with Logging {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@ -61,7 +62,7 @@ class NewHadoopRDD[K, V](
reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
// Register an on-task-completion callback to close the input stream. // Register an on-task-completion callback to close the input stream.
context.addOnCompleteCallback(() => reader.close()) context.addOnCompleteCallback(() => close())
var havePair = false var havePair = false
var finished = false var finished = false
@ -81,6 +82,14 @@ class NewHadoopRDD[K, V](
havePair = false havePair = false
return (reader.getCurrentKey, reader.getCurrentValue) return (reader.getCurrentKey, reader.getCurrentValue)
} }
private def close() {
try {
reader.close()
} catch {
case e: Exception => logWarning("Exception in RecordReader.close()", e)
}
}
} }
override def getPreferredLocations(split: Partition): Seq[String] = { override def getPreferredLocations(split: Partition): Seq[String] = {

View file

@ -0,0 +1,108 @@
package spark.rdd
import java.util.{HashSet => JHashSet}
import scala.collection.JavaConversions._
import spark.RDD
import spark.Partitioner
import spark.Dependency
import spark.TaskContext
import spark.Partition
import spark.SparkEnv
import spark.ShuffleDependency
import spark.OneToOneDependency
/**
* An optimized version of cogroup for set difference/subtraction.
*
* It is possible to implement this operation with just `cogroup`, but
* that is less efficient because all of the entries from `rdd2`, for
* both matching and non-matching values in `rdd1`, are kept in the
* JHashMap until the end.
*
* With this implementation, only the entries from `rdd1` are kept in-memory,
* and the entries from `rdd2` are essentially streamed, as we only need to
* touch each once to decide if the value needs to be removed.
*
* This is particularly helpful when `rdd1` is much smaller than `rdd2`, as
* 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[T: ClassManifest](
@transient var rdd1: RDD[T],
@transient var rdd2: RDD[T],
part: Partitioner) extends RDD[T](rdd1.context, Nil) {
override def getDependencies: Seq[Dependency[_]] = {
Seq(rdd1, rdd2).map { rdd =>
if (rdd.partitioner == Some(part)) {
logInfo("Adding one-to-one dependency with " + rdd)
new OneToOneDependency(rdd)
} else {
logInfo("Adding shuffle dependency with " + rdd)
val mapSideCombinedRDD = rdd.mapPartitions(i => {
val set = new JHashSet[T]()
while (i.hasNext) {
set.add(i.next)
}
set.iterator
}, true)
// ShuffleDependency requires a tuple (k, v), which it will partition by k.
// We need this to partition to map to the same place as the k for
// OneToOneDependency, which means:
// - for already-tupled RDD[(A, B)], into getPartition(a)
// - for non-tupled RDD[C], into getPartition(c)
val part2 = new Partitioner() {
def numPartitions = part.numPartitions
def getPartition(key: Any) = key match {
case (k, v) => part.getPartition(k)
case k => part.getPartition(k)
}
}
new ShuffleDependency(mapSideCombinedRDD.map((_, null)), part2)
}
}
}
override def getPartitions: Array[Partition] = {
val array = new Array[Partition](part.numPartitions)
for (i <- 0 until array.size) {
// Each CoGroupPartition will depend on rdd1 and rdd2
array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) =>
dependencies(j) match {
case s: ShuffleDependency[_, _] =>
new ShuffleCoGroupSplitDep(s.shuffleId)
case _ =>
new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
}
}.toList)
}
array
}
override val partitioner = Some(part)
override def compute(p: Partition, context: TaskContext): Iterator[T] = {
val partition = p.asInstanceOf[CoGroupPartition]
val set = new JHashSet[T]
def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match {
case NarrowCoGroupSplitDep(rdd, _, itsSplit) =>
for (k <- rdd.iterator(itsSplit, context))
op(k.asInstanceOf[T])
case ShuffleCoGroupSplitDep(shuffleId) =>
for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index))
op(k.asInstanceOf[T])
}
// the first dep is rdd1; add all keys to the set
integrate(partition.deps(0), set.add)
// the second dep is rdd2; remove all of its keys from the set
integrate(partition.deps(1), set.remove)
set.iterator
}
override def clearDependencies() {
super.clearDependencies()
rdd1 = null
rdd2 = null
}
}

View file

@ -11,6 +11,7 @@ import spark.TaskState.TaskState
import spark.scheduler._ import spark.scheduler._
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.{TimerTask, Timer}
/** /**
* The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
@ -22,6 +23,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
// How often to check for speculative tasks // How often to check for speculative tasks
val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong val SPECULATION_INTERVAL = System.getProperty("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 activeTaskSets = new HashMap[String, TaskSetManager] val activeTaskSets = new HashMap[String, TaskSetManager]
var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager]
@ -30,6 +33,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
val taskIdToExecutorId = new HashMap[Long, String] val taskIdToExecutorId = new HashMap[Long, String]
val taskSetTaskIds = new HashMap[String, HashSet[Long]] val taskSetTaskIds = new HashMap[String, HashSet[Long]]
var hasReceivedTask = false
var hasLaunchedTask = false
val starvationTimer = new Timer(true)
// Incrementing Mesos task IDs // Incrementing Mesos task IDs
val nextTaskId = new AtomicLong(0) val nextTaskId = new AtomicLong(0)
@ -94,6 +101,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
activeTaskSets(taskSet.id) = manager activeTaskSets(taskSet.id) = manager
activeTaskSetsQueue += manager activeTaskSetsQueue += manager
taskSetTaskIds(taskSet.id) = new HashSet[Long]() taskSetTaskIds(taskSet.id) = new HashSet[Long]()
if (hasReceivedTask == false) {
starvationTimer.scheduleAtFixedRate(new TimerTask() {
override def run() {
if (!hasLaunchedTask) {
logWarning("Initial job has not accepted any resources; " +
"check your cluster UI to ensure that workers are registered")
} else {
this.cancel()
}
}
}, STARVATION_TIMEOUT, STARVATION_TIMEOUT)
}
hasReceivedTask = true;
} }
backend.reviveOffers() backend.reviveOffers()
} }
@ -119,6 +140,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
// Mark each slave as alive and remember its hostname // Mark each slave as alive and remember its hostname
for (o <- offers) { for (o <- offers) {
executorIdToHost(o.executorId) = o.hostname executorIdToHost(o.executorId) = o.hostname
if (!executorsByHost.contains(o.hostname)) {
executorsByHost(o.hostname) = new HashSet()
}
} }
// Build a list of tasks to assign to each slave // Build a list of tasks to assign to each slave
val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
@ -138,9 +162,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
taskSetTaskIds(manager.taskSet.id) += tid taskSetTaskIds(manager.taskSet.id) += tid
taskIdToExecutorId(tid) = execId taskIdToExecutorId(tid) = execId
activeExecutorIds += execId activeExecutorIds += execId
if (!executorsByHost.contains(host)) {
executorsByHost(host) = new HashSet()
}
executorsByHost(host) += execId executorsByHost(host) += execId
availableCpus(i) -= 1 availableCpus(i) -= 1
launchedTask = true launchedTask = true
@ -150,6 +171,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
} }
} while (launchedTask) } while (launchedTask)
} }
if (tasks.size > 0) {
hasLaunchedTask = true
}
return tasks return tasks
} }
} }
@ -235,7 +259,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
} }
override def defaultParallelism() = backend.defaultParallelism() override def defaultParallelism() = backend.defaultParallelism()
// Check for speculatable tasks in all our active jobs. // Check for speculatable tasks in all our active jobs.
def checkSpeculatableTasks() { def checkSpeculatableTasks() {
var shouldRevive = false var shouldRevive = false

View file

@ -153,7 +153,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
driverActor ! ReviveOffers driverActor ! ReviveOffers
} }
override def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
.map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
// Called by subclasses when notified of a lost worker // Called by subclasses when notified of a lost worker
def removeExecutor(executorId: String, reason: String) { def removeExecutor(executorId: String, reason: String) {

View file

@ -26,7 +26,7 @@ private[spark] class BlockManagerMaster(
val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt 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_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
val timeout = 10.seconds val timeout = 10.seconds
var driverActor: ActorRef = { var driverActor: ActorRef = {

View file

@ -32,8 +32,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
} }
override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
bytes.rewind()
if (level.deserialized) { if (level.deserialized) {
bytes.rewind()
val values = blockManager.dataDeserialize(blockId, bytes) val values = blockManager.dataDeserialize(blockId, bytes)
val elements = new ArrayBuffer[Any] val elements = new ArrayBuffer[Any]
elements ++= values elements ++= values
@ -58,7 +58,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
} else { } else {
val bytes = blockManager.dataSerialize(blockId, values.iterator) val bytes = blockManager.dataSerialize(blockId, values.iterator)
tryToPut(blockId, bytes, bytes.limit, false) tryToPut(blockId, bytes, bytes.limit, false)
PutResult(bytes.limit(), Right(bytes)) PutResult(bytes.limit(), Right(bytes.duplicate()))
} }
} }

View file

@ -31,20 +31,22 @@ private[spark] object AkkaUtils {
val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt
val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
val akkaConf = ConfigFactory.parseString(""" val akkaConf = ConfigFactory.parseString("""
akka.daemonic = on akka.daemonic = on
akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
akka.stdout-loglevel = "ERROR" akka.stdout-loglevel = "ERROR"
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = "akka.remote.RemoteActorRefProvider"
akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
akka.remote.log-remote-lifecycle-events = on
akka.remote.netty.hostname = "%s" akka.remote.netty.hostname = "%s"
akka.remote.netty.port = %d akka.remote.netty.port = %d
akka.remote.netty.connection-timeout = %ds akka.remote.netty.connection-timeout = %ds
akka.remote.netty.message-frame-size = %d MiB akka.remote.netty.message-frame-size = %d MiB
akka.remote.netty.execution-pool-size = %d akka.remote.netty.execution-pool-size = %d
akka.actor.default-dispatcher.throughput = %d akka.actor.default-dispatcher.throughput = %d
""".format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize)) akka.remote.log-remote-lifecycle-events = %s
""".format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
if (lifecycleEvents) "on" else "off"))
val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader)

View file

@ -38,7 +38,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
object MetadataCleaner { object MetadataCleaner {
def getDelaySeconds = System.getProperty("spark.cleaner.delay", "-1").toInt def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt
def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.delay", delay.toString) } def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) }
} }

View file

@ -11,12 +11,16 @@ class Vector(val elements: Array[Double]) extends Serializable {
return Vector(length, i => this(i) + other(i)) return Vector(length, i => this(i) + other(i))
} }
def add(other: Vector) = this + other
def - (other: Vector): Vector = { def - (other: Vector): Vector = {
if (length != other.length) if (length != other.length)
throw new IllegalArgumentException("Vectors of different length") throw new IllegalArgumentException("Vectors of different length")
return Vector(length, i => this(i) - other(i)) return Vector(length, i => this(i) - other(i))
} }
def subtract(other: Vector) = this - other
def dot(other: Vector): Double = { def dot(other: Vector): Double = {
if (length != other.length) if (length != other.length)
throw new IllegalArgumentException("Vectors of different length") throw new IllegalArgumentException("Vectors of different length")
@ -61,10 +65,16 @@ class Vector(val elements: Array[Double]) extends Serializable {
this this
} }
def addInPlace(other: Vector) = this +=other
def * (scale: Double): Vector = Vector(length, i => this(i) * scale) def * (scale: Double): Vector = Vector(length, i => this(i) * scale)
def multiply (d: Double) = this * d
def / (d: Double): Vector = this * (1 / d) def / (d: Double): Vector = this * (1 / d)
def divide (d: Double) = this / d
def unary_- = this * -1 def unary_- = this * -1
def sum = elements.reduceLeft(_ + _) def sum = elements.reduceLeft(_ + _)

View file

@ -162,6 +162,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
} }
test("CheckpointRDD with zero partitions") {
val rdd = new BlockRDD[Int](sc, Array[String]())
assert(rdd.partitions.size === 0)
assert(rdd.isCheckpointed === false)
rdd.checkpoint()
assert(rdd.count() === 0)
assert(rdd.isCheckpointed === true)
assert(rdd.partitions.size === 0)
}
/** /**
* Test checkpointing of the final RDD generated by the given operation. By default, * Test checkpointing of the final RDD generated by the given operation. By default,
* this method tests whether the size of serialized RDD has reduced after checkpointing or not. * this method tests whether the size of serialized RDD has reduced after checkpointing or not.

View file

@ -1,5 +1,6 @@
package spark package spark
import network.ConnectionManagerId
import org.scalatest.FunSuite import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import org.scalatest.matchers.ShouldMatchers import org.scalatest.matchers.ShouldMatchers
@ -13,7 +14,7 @@ import com.google.common.io.Files
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import SparkContext._ import SparkContext._
import storage.StorageLevel import storage.{GetBlock, BlockManagerWorker, StorageLevel}
class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext {
@ -140,9 +141,22 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
test("caching in memory and disk, serialized, replicated") { test("caching in memory and disk, serialized, replicated") {
sc = new SparkContext(clusterUrl, "test") sc = new SparkContext(clusterUrl, "test")
val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2)
assert(data.count() === 1000) assert(data.count() === 1000)
assert(data.count() === 1000) assert(data.count() === 1000)
assert(data.count() === 1000) assert(data.count() === 1000)
// Get all the locations of the first partition and try to fetch the partitions
// from those locations.
val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray
val blockId = blockIds(0)
val blockManager = SparkEnv.get.blockManager
blockManager.master.getLocations(blockId).foreach(id => {
val bytes = BlockManagerWorker.syncGetBlock(
GetBlock(blockId), ConnectionManagerId(id.ip, id.port))
val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList
assert(deserialized === (1 to 100).toList)
})
} }
test("compute without caching when no partitions fit in memory") { test("compute without caching when no partitions fit in memory") {
@ -217,6 +231,27 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
assert(grouped.collect.size === 1) assert(grouped.collect.size === 1)
} }
} }
test("recover from node failures with replication") {
import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity}
DistributedSuite.amMaster = true
// Using more than two nodes so we don't have a symmetric communication pattern and might
// cache a partially correct list of peers.
sc = new SparkContext("local-cluster[3,1,512]", "test")
for (i <- 1 to 3) {
val data = sc.parallelize(Seq(true, false, false, false), 4)
data.persist(StorageLevel.MEMORY_ONLY_2)
assert(data.count === 4)
assert(data.map(markNodeIfIdentity).collect.size === 4)
assert(data.map(failOnMarkedIdentity).collect.size === 4)
// Create a new replicated RDD to make sure that cached peer information doesn't cause
// problems.
val data2 = sc.parallelize(Seq(true, true), 2).persist(StorageLevel.MEMORY_ONLY_2)
assert(data2.count === 2)
}
}
} }
object DistributedSuite { object DistributedSuite {

View file

@ -84,10 +84,10 @@ class PartitioningSuite extends FunSuite with LocalSparkContext {
assert(grouped4.groupByKey(3).partitioner != grouped4.partitioner) assert(grouped4.groupByKey(3).partitioner != grouped4.partitioner)
assert(grouped4.groupByKey(4).partitioner === grouped4.partitioner) assert(grouped4.groupByKey(4).partitioner === grouped4.partitioner)
assert(grouped2.join(grouped4).partitioner === grouped2.partitioner) assert(grouped2.join(grouped4).partitioner === grouped4.partitioner)
assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped2.partitioner) assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped4.partitioner)
assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped2.partitioner) assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped4.partitioner)
assert(grouped2.cogroup(grouped4).partitioner === grouped2.partitioner) assert(grouped2.cogroup(grouped4).partitioner === grouped4.partitioner)
assert(grouped2.join(reduced2).partitioner === grouped2.partitioner) assert(grouped2.join(reduced2).partitioner === grouped2.partitioner)
assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner)

View file

@ -234,6 +234,51 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.keys.collect().toList === List(1, 2))
assert(rdd.values.collect().toList === List("a", "b")) assert(rdd.values.collect().toList === List("a", "b"))
} }
test("default partitioner uses partition size") {
sc = new SparkContext("local", "test")
// specify 2000 partitions
val a = sc.makeRDD(Array(1, 2, 3, 4), 2000)
// do a map, which loses the partitioner
val b = a.map(a => (a, (a * 2).toString))
// then a group by, and see we didn't revert to 2 partitions
val c = b.groupByKey()
assert(c.partitions.size === 2000)
}
test("default partitioner uses largest partitioner") {
sc = new SparkContext("local", "test")
val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2)
val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000)
val c = a.join(b)
assert(c.partitions.size === 2000)
}
test("subtract") {
sc = new SparkContext("local", "test")
val a = sc.parallelize(Array(1, 2, 3), 2)
val b = sc.parallelize(Array(2, 3, 4), 4)
val c = a.subtract(b)
assert(c.collect().toSet === Set(1))
assert(c.partitions.size === a.partitions.size)
}
test("subtract with narrow dependency") {
sc = new SparkContext("local", "test")
// use a deterministic partitioner
val p = new Partitioner() {
def numPartitions = 5
def getPartition(key: Any) = key.asInstanceOf[Int]
}
// partitionBy so we have a narrow dependency
val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p)
println(sc.runJob(a, (i: Iterator[(Int, String)]) => i.toList).toList)
// more partitions/no partitioner so a shuffle dependency
val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4)
val c = a.subtract(b)
assert(c.collect().toSet === Set((1, "a"), (3, "c")))
assert(c.partitioner.get === p)
}
} }
object ShuffleSuite { object ShuffleSuite {

View file

@ -33,4 +33,4 @@ Similarly, you can build just the PySpark epydoc by running `epydoc --config epy
When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/).
NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`. Similarly, `SKIP_EPYDOC=1 jekyll` will skip PySpark API doc generation. NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`.

View file

@ -3,8 +3,8 @@ markdown: kramdown
# These allow the documentation to be updated with nerw releases # These allow the documentation to be updated with nerw releases
# of Spark, Scala, and Mesos. # of Spark, Scala, and Mesos.
SPARK_VERSION: 0.7.0-SNAPSHOT SPARK_VERSION: 0.7.1-SNAPSHOT
SPARK_VERSION_SHORT: 0.7.0 SPARK_VERSION_SHORT: 0.7.1
SCALA_VERSION: 2.9.2 SCALA_VERSION: 2.9.2
MESOS_VERSION: 0.9.0-incubating MESOS_VERSION: 0.9.0-incubating
SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net

View file

@ -23,6 +23,22 @@
<script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script> <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="css/pygments-default.css"> <link rel="stylesheet" href="css/pygments-default.css">
<!-- Google analytics script -->
<script type="text/javascript">
/*
var _gaq = _gaq || [];
_gaq.push(['_setAccount', 'UA-32518208-1']);
_gaq.push(['_trackPageview']);
(function() {
var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
})();
*/
</script>
</head> </head>
<body> <body>
<!--[if lt IE 7]> <!--[if lt IE 7]>
@ -53,11 +69,11 @@
</li> </li>
<li class="dropdown"> <li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">API (Scaladoc)<b class="caret"></b></a> <a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a>
<ul class="dropdown-menu"> <ul class="dropdown-menu">
<li><a href="api/core/index.html">Spark Scala/Java (Scaladoc)</a></li> <li><a href="api/core/index.html">Spark Java/Scala (Scaladoc)</a></li>
<li><a href="api/pyspark/index.html">Spark Python (Epydoc)</a></li> <li><a href="api/pyspark/index.html">Spark Python (Epydoc)</a></li>
<li><a href="api/streaming/index.html">Spark Streaming Scala/Java (Scaladoc) </a></li> <li><a href="api/streaming/index.html">Spark Streaming Java/Scala (Scaladoc) </a></li>
</ul> </ul>
</li> </li>

View file

@ -1,7 +1,8 @@
require 'fileutils' require 'fileutils'
include FileUtils include FileUtils
if ENV['SKIP_SCALADOC'] != '1' if ENV['SKIP_API'] != '1'
# Build Scaladoc for Java/Scala
projects = ["core", "examples", "repl", "bagel", "streaming"] projects = ["core", "examples", "repl", "bagel", "streaming"]
puts "Moving to project root and building scaladoc." puts "Moving to project root and building scaladoc."
@ -27,9 +28,8 @@ if ENV['SKIP_SCALADOC'] != '1'
puts "cp -r " + source + "/. " + dest puts "cp -r " + source + "/. " + dest
cp_r(source + "/.", dest) cp_r(source + "/.", dest)
end end
end
if ENV['SKIP_EPYDOC'] != '1' # Build Epydoc for Python
puts "Moving to python directory and building epydoc." puts "Moving to python directory and building epydoc."
cd("../python") cd("../python")
puts `epydoc --config epydoc.conf` puts `epydoc --config epydoc.conf`

View file

@ -133,6 +133,13 @@ Apart from these, the following properties are also available, and may be useful
it if you configure your own old generation size. it if you configure your own old generation size.
</td> </td>
</tr> </tr>
<tr>
<td>spark.ui.port</td>
<td>(random)</td>
<td>
Port for your application's dashboard, which shows memory usage of each RDD.
</td>
</tr>
<tr> <tr>
<td>spark.shuffle.compress</td> <td>spark.shuffle.compress</td>
<td>true</td> <td>true</td>
@ -183,7 +190,7 @@ Apart from these, the following properties are also available, and may be useful
</tr> </tr>
<tr> <tr>
<td>spark.broadcast.factory</td> <td>spark.broadcast.factory</td>
<td>spark.broadcast. HttpBroadcastFactory</td> <td>spark.broadcast.HttpBroadcastFactory</td>
<td> <td>
Which broadcast implementation to use. Which broadcast implementation to use.
</td> </td>
@ -244,10 +251,10 @@ Apart from these, the following properties are also available, and may be useful
</td> </td>
</tr> </tr>
<tr> <tr>
<td>spark.cleaner.delay</td> <td>spark.cleaner.ttl</td>
<td>(disable)</td> <td>(disable)</td>
<td> <td>
Duration (minutes) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is
useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming
applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. applications). Note that any RDD that persists in memory for more than this duration will be cleared as well.

View file

@ -2719,16 +2719,16 @@ table .span24 {
.dropdown-menu .active > a:hover { .dropdown-menu .active > a:hover {
color: #ffffff; color: #ffffff;
text-decoration: none; text-decoration: none;
background-color: #0088cc; background-color: #0098cc;
background-color: #0081c2; background-color: #0098cc;
background-image: linear-gradient(to bottom, #0088cc, #0077b3); background-image: linear-gradient(to bottom, #0098cc, #0087b3);
background-image: -moz-linear-gradient(top, #0088cc, #0077b3); background-image: -moz-linear-gradient(top, #0098cc, #0087b3);
background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3)); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0098cc), to(#0087b3));
background-image: -webkit-linear-gradient(top, #0088cc, #0077b3); background-image: -webkit-linear-gradient(top, #0098cc, #0087b3);
background-image: -o-linear-gradient(top, #0088cc, #0077b3); background-image: -o-linear-gradient(top, #0098cc, #0087b3);
background-repeat: repeat-x; background-repeat: repeat-x;
outline: 0; outline: 0;
filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0); filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc', endColorstr='#ff0087b3', GradientType=0);
} }
.dropdown-menu .disabled > a, .dropdown-menu .disabled > a,
@ -4025,18 +4025,18 @@ input[type="submit"].btn.btn-mini {
min-height: 40px; min-height: 40px;
padding-right: 20px; padding-right: 20px;
padding-left: 20px; padding-left: 20px;
background-color: #fafafa; background-color: #fadafa;
background-image: -moz-linear-gradient(top, #ffffff, #f2f2f2); background-image: -moz-linear-gradient(top, #ffddff, #f2d2f2);
background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2)); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffddff), to(#f2d2f2));
background-image: -webkit-linear-gradient(top, #ffffff, #f2f2f2); background-image: -webkit-linear-gradient(top, #ffddff, #f2d2f2);
background-image: -o-linear-gradient(top, #ffffff, #f2f2f2); background-image: -o-linear-gradient(top, #ffddff, #f2d2f2);
background-image: linear-gradient(to bottom, #ffffff, #f2f2f2); background-image: linear-gradient(to bottom, #ffddff, #f2d2f2);
background-repeat: repeat-x; background-repeat: repeat-x;
border: 1px solid #d4d4d4; border: 1px solid #d4d4d4;
-webkit-border-radius: 4px; -webkit-border-radius: 4px;
-moz-border-radius: 4px; -moz-border-radius: 4px;
border-radius: 4px; border-radius: 4px;
filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0); filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff', endColorstr='#fff2d2f2', GradientType=0);
-webkit-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); -webkit-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);
-moz-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); -moz-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);
box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);

File diff suppressed because one or more lines are too long

View file

@ -45,9 +45,9 @@ identify machines belonging to each cluster in the Amazon EC2 Console.
key pair, `<num-slaves>` is the number of slave nodes to launch (try key pair, `<num-slaves>` is the number of slave nodes to launch (try
1 at first), and `<cluster-name>` is the name to give to your 1 at first), and `<cluster-name>` is the name to give to your
cluster. cluster.
- After everything launches, check that Mesos is up and sees all the - After everything launches, check that the cluster scheduler is up and sees
slaves by going to the Mesos Web UI link printed at the end of the all the slaves by going to its web UI, which will be printed at the end of
script (`http://<master-hostname>:8080`). the script (typically `http://<master-hostname>:8080`).
You can also run `./spark-ec2 --help` to see more usage options. The You can also run `./spark-ec2 --help` to see more usage options. The
following options are worth pointing out: following options are worth pointing out:
@ -68,6 +68,9 @@ available.
- `--ebs-vol-size=GB` will attach an EBS volume with a given amount - `--ebs-vol-size=GB` will attach an EBS volume with a given amount
of space to each node so that you can have a persistent HDFS cluster of space to each node so that you can have a persistent HDFS cluster
on your nodes across cluster restarts (see below). on your nodes across cluster restarts (see below).
- `--spot-price=PRICE` will launch the worker nodes as
[Spot Instances](http://aws.amazon.com/ec2/spot-instances/),
bidding for the given maximum price (in dollars).
- If one of your launches fails due to e.g. not having the right - If one of your launches fails due to e.g. not having the right
permissions on your private key file, you can run `launch` with the permissions on your private key file, you can run `launch` with the
`--resume` option to restart the setup process on an existing cluster. `--resume` option to restart the setup process on an existing cluster.
@ -80,7 +83,7 @@ permissions on your private key file, you can run `launch` with the
above. (This is just for convenience; you could also use above. (This is just for convenience; you could also use
the EC2 console.) the EC2 console.)
- To deploy code or data within your cluster, you can log in and use the - To deploy code or data within your cluster, you can log in and use the
provided script `~/mesos-ec2/copy-dir`, which, provided script `~/spark-ec2/copy-dir`, which,
given a directory path, RSYNCs it to the same location on all the slaves. given a directory path, RSYNCs it to the same location on all the slaves.
- If your job needs to access large datasets, the fastest way to do - If your job needs to access large datasets, the fastest way to do
that is to load them from Amazon S3 or an Amazon EBS device into an that is to load them from Amazon S3 or an Amazon EBS device into an
@ -106,7 +109,7 @@ You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark config
as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`). as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`).
This file needs to be copied to **every machine** to reflect the change. The easiest way to do this This file needs to be copied to **every machine** to reflect the change. The easiest way to do this
is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master,
then run `~/mesos-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers.
The [configuration guide](configuration.html) describes the available configuration options. The [configuration guide](configuration.html) describes the available configuration options.
@ -152,10 +155,10 @@ If you have a patch or suggestion for one of these limitations, feel free to
# Using a Newer Spark Version # Using a Newer Spark Version
The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/mesos-ec2/copy-dir /root/spark`. The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/spark-ec2/copy-dir /root/spark`.
# Accessing Data in S3 # Accessing Data in S3
Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<id>:<secret>@<bucket>/path`, where `<id>` is your Amazon access key ID and `<secret>` is your Amazon secret access key. Note that you should escape any `/` characters in the secret key as `%2F`. Full instructions can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<bucket>/path`. You will also need to set your Amazon security credentials, either by setting the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` before your program or through `SparkContext.hadoopConfiguration`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3).
In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory. In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory.

View file

@ -3,15 +3,9 @@ layout: global
title: Spark Overview title: Spark Overview
--- ---
{% comment %}
TODO(andyk): Rewrite to make the Java API a first class part of the story.
{% endcomment %}
Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter.
It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators. It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators.
Spark can run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager, Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode").
[Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html),
Amazon EC2, or without an independent resource manager ("standalone mode").
# Downloading # Downloading
@ -58,9 +52,9 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
* [Quick Start](quick-start.html): a quick introduction to the Spark API; start here! * [Quick Start](quick-start.html): a quick introduction to the Spark API; start here!
* [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API
* [Streaming Programming Guide](streaming-programming-guide.html): an API preview of Spark Streaming
* [Java Programming Guide](java-programming-guide.html): using Spark from Java * [Java Programming Guide](java-programming-guide.html): using Spark from Java
* [Python Programming Guide](python-programming-guide.html): using Spark from Python * [Python Programming Guide](python-programming-guide.html): using Spark from Python
* [Spark Streaming Guide](streaming-programming-guide.html): using the alpha release of Spark Streaming
**API Docs:** **API Docs:**
@ -92,7 +86,8 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
[slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are
available online for free. available online for free.
* [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark * [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark
* [Paper Describing the Spark System](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf)
* [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
# Community # Community

View file

@ -189,7 +189,7 @@ We hope to generate documentation with Java-style syntax in the future.
# Where to Go from Here # Where to Go from Here
Spark includes several sample programs using the Java API in Spark includes several sample programs using the Java API in
`examples/src/main/java`. You can run them by passing the class name to the [`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the
`run` script included in Spark -- for example, `./run `run` script included in Spark -- for example, `./run
spark.examples.JavaWordCount`. Each example program prints usage help when run spark.examples.JavaWordCount`. Each example program prints usage help when run
without any arguments. without any arguments.

View file

@ -87,7 +87,7 @@ By default, the `pyspark` shell creates SparkContext that runs jobs locally.
To connect to a non-local cluster, set the `MASTER` environment variable. To connect to a non-local cluster, set the `MASTER` environment variable.
For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
{% highlight shell %} {% highlight bash %}
$ MASTER=spark://IP:PORT ./pyspark $ MASTER=spark://IP:PORT ./pyspark
{% endhighlight %} {% endhighlight %}
@ -109,9 +109,9 @@ Code dependencies can be added to an existing SparkContext using its `addPyFile(
# Where to Go from Here # Where to Go from Here
PySpark includes several sample programs using the Python API in `python/examples`. PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples).
You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`. You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`.
Each example program prints usage help when run without any arguments. Each program prints usage help when run without arguments.
We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc.
Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples.

View file

@ -136,7 +136,7 @@ scalaVersion := "{{site.SCALA_VERSION}}"
libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}" libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}"
resolvers ++= Seq( resolvers ++= Seq(
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", "Akka Repository" at "http://repo.akka.io/releases/",
"Spray Repository" at "http://repo.spray.cc/") "Spray Repository" at "http://repo.spray.cc/")
{% endhighlight %} {% endhighlight %}
@ -207,8 +207,8 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep
<url>http://repo.spray.cc</url> <url>http://repo.spray.cc</url>
</repository> </repository>
<repository> <repository>
<id>Typesafe repository</id> <id>Akka repository</id>
<url>http://repo.typesafe.com/typesafe/releases</url> <url>http://repo.akka.io/releases</url>
</repository> </repository>
</repositories> </repositories>
<dependencies> <dependencies>

View file

@ -38,10 +38,10 @@ The first thing a Spark program must do is to create a `SparkContext` object, wh
This is done through the following constructor: This is done through the following constructor:
{% highlight scala %} {% highlight scala %}
new SparkContext(master, jobName, [sparkHome], [jars]) new SparkContext(master, appName, [sparkHome], [jars])
{% endhighlight %} {% endhighlight %}
The `master` parameter is a string specifying a [Mesos](running-on-mesos.html) cluster to connect to, or a special "local" string to run in local mode, as described below. `jobName` is a name for your job, which will be shown in the Mesos web UI when running on a cluster. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use

View file

@ -0,0 +1,101 @@
---
layout: global
title: Tutorial - Spark Streaming, Plugging in a custom receiver.
---
A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need.
This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application.
## A quick and naive walk-through
### Write a simple receiver
This starts with implementing [Actor](#References)
Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api.
{% highlight scala %}
class SocketTextStreamReceiver (host:String,
port:Int,
bytesToString: ByteString => String) extends Actor with Receiver {
override def preStart = IOManager(context.system).connect(host, port)
def receive = {
case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes))
}
}
{% endhighlight %}
All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details.
### A sample spark application
* First create a Spark streaming context with master url and batchduration.
{% highlight scala %}
val ssc = new StreamingContext(master, "WordCountCustomStreamSource",
Seconds(batchDuration))
{% endhighlight %}
* Plug-in the actor configuration into the spark streaming context and create a DStream.
{% highlight scala %}
val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
"localhost",8445, z => z.utf8String)),"SocketReceiver")
{% endhighlight %}
* Process it.
{% highlight scala %}
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
{% endhighlight %}
* After processing it, stream can be tested using the netcat utility.
$ nc -l localhost 8445
hello world
hello hello
## Multiple homogeneous/heterogeneous receivers.
A DStream union operation is provided for taking union on multiple input streams.
{% highlight scala %}
val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
"localhost",8445, z => z.utf8String)),"SocketReceiver")
// Another socket stream receiver
val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
"localhost",8446, z => z.utf8String)),"SocketReceiver")
val union = lines.union(lines2)
{% endhighlight %}
Above stream can be easily process as described earlier.
_A more comprehensive example is provided in the spark streaming examples_
## References
1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)

View file

@ -7,9 +7,9 @@ title: Spark Streaming Programming Guide
{:toc} {:toc}
# Overview # Overview
A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collection of elements) representing a continuous stream of data. DStreams can created from live incoming data (such as data from a socket, Kafka, etc.) or it can be generated by transformation of existing DStreams using parallel operators like map, reduce, and window. The basic processing model is as follows: A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows:
(i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides the data into batches. Each batch of data is treated as a RDD, that is a immutable and parallel collection of data. These input data RDDs are automatically persisted in memory (serialized by default) and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively referred to as an InputDStream. (i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream.
(ii) Data received by InputDStreams are processed processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. (ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures.
This guide shows some how to start programming with DStreams. This guide shows some how to start programming with DStreams.
@ -17,16 +17,12 @@ This guide shows some how to start programming with DStreams.
The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using
{% highlight scala %} {% highlight scala %}
new StreamingContext(master, jobName, batchDuration) new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
{% endhighlight %} {% endhighlight %}
The `master` parameter is either the [Mesos master URL](running-on-mesos.html) (for running on a cluster)or the special "local" string (for local mode) that is used to create a Spark Context. For more information about this please refer to the [Spark programming guide](scala-programming-guide.html). The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the Mesos web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds maybe a good start. See [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using
{% highlight scala %}
new StreamingContext(sparkContext, batchDuration)
{% endhighlight %}
This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`.
# Attaching Input Sources - InputDStreams # Attaching Input Sources - InputDStreams
@ -34,34 +30,34 @@ The StreamingContext is used to creating InputDStreams from input sources:
{% highlight scala %} {% highlight scala %}
// Assuming ssc is the StreamingContext // Assuming ssc is the StreamingContext
ssc.networkStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory
ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port
{% endhighlight %} {% endhighlight %}
A complete list of input sources is available in the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). Data received from these sources can be processed using DStream operations, which are explained next. We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext).
# DStream Operations # DStream Operations
Once an input DStream has been created, you can transform it using _DStream operators_. Most of these operators return new DStreams which you can further transform. Eventually, you'll need to call an _output operator_, which forces evaluation of the DStream by writing data out to an external source. Data received from the input streams can be processed using _DStream operations_. There are two kinds of operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams with transformed data. After applying a sequence of transformations to the input streams, you'll need to call the output operations, which writies data out to an external source.
## Transformations ## Transformations
DStreams support many of the transformations available on normal Spark RDD's: DStreams support many of the transformations available on normal Spark RDD's:
<table class="table"> <table class="table">
<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr> <tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
<tr> <tr>
<td> <b>map</b>(<i>func</i>) </td> <td> <b>map</b>(<i>func</i>) </td>
<td> Returns a new DStream formed by passing each element of the source through a function <i>func</i>. </td> <td> Returns a new DStream formed by passing each element of the source DStream through a function <i>func</i>. </td>
</tr> </tr>
<tr> <tr>
<td> <b>filter</b>(<i>func</i>) </td> <td> <b>filter</b>(<i>func</i>) </td>
<td> Returns a new stream formed by selecting those elements of the source on which <i>func</i> returns true. </td> <td> Returns a new DStream formed by selecting those elements of the source DStream on which <i>func</i> returns true. </td>
</tr> </tr>
<tr> <tr>
<td> <b>flatMap</b>(<i>func</i>) </td> <td> <b>flatMap</b>(<i>func</i>) </td>
<td> Similar to map, but each input item can be mapped to 0 or more output items (so <i>func</i> should return a Seq rather than a single item). </td> <td> Similar to map, but each input item can be mapped to 0 or more output items (so <i>func</i> should return a <code>Seq</code> rather than a single item). </td>
</tr> </tr>
<tr> <tr>
<td> <b>mapPartitions</b>(<i>func</i>) </td> <td> <b>mapPartitions</b>(<i>func</i>) </td>
@ -70,73 +66,92 @@ DStreams support many of the transformations available on normal Spark RDD's:
</tr> </tr>
<tr> <tr>
<td> <b>union</b>(<i>otherStream</i>) </td> <td> <b>union</b>(<i>otherStream</i>) </td>
<td> Return a new stream that contains the union of the elements in the source stream and the argument. </td> <td> Return a new DStream that contains the union of the elements in the source DStream and the argument DStream. </td>
</tr>
<tr>
<td> <b>count</b>() </td>
<td> Returns a new DStream of single-element RDDs by counting the number of elements in each RDD of the source DStream. </td>
</tr>
<tr>
<td> <b>reduce</b>(<i>func</i>) </td>
<td> Returns a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function <i>func</i> (which takes two arguments and returns one). The function should be associative so that it can be computed in parallel. </td>
</tr>
<tr>
<td> <b>countByValue</b>() </td>
<td> When called on a DStream of elements of type K, returns a new DStream of (K, Long) pairs where the value of each key is its frequency in each RDD of the source DStream. </td>
</tr> </tr>
<tr> <tr>
<td> <b>groupByKey</b>([<i>numTasks</i>]) </td> <td> <b>groupByKey</b>([<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs. <br /> <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together all the values of each key in the RDDs of the source DStream. <br />
<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. <b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
</td> </td>
</tr> </tr>
<tr> <tr>
<td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td> <td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in <code>groupByKey</code>, the number of reduce tasks is configurable through an optional second argument. </td> <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in <code>groupByKey</code>, the number of reduce tasks is configurable through an optional second argument. </td>
</tr> </tr>
<tr> <tr>
<td> <b>join</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td> <td> <b>join</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
<td> When called on streams of type (K, V) and (K, W), returns a stream of (K, (V, W)) pairs with all pairs of elements for each key. </td> <td> When called on two DStreams of (K, V) and (K, W) pairs, returns a new DStream of (K, (V, W)) pairs with all pairs of elements for each key. </td>
</tr> </tr>
<tr> <tr>
<td> <b>cogroup</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td> <td> <b>cogroup</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
<td> When called on DStream of type (K, V) and (K, W), returns a DStream of (K, Seq[V], Seq[W]) tuples.</td> <td> When called on DStream of (K, V) and (K, W) pairs, returns a new DStream of (K, Seq[V], Seq[W]) tuples.</td>
</tr>
<tr>
<td> <b>reduce</b>(<i>func</i>) </td>
<td> Returns a new DStream of single-element RDDs by aggregating the elements of the stream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. </td>
</tr> </tr>
<tr> <tr>
<td> <b>transform</b>(<i>func</i>) </td> <td> <b>transform</b>(<i>func</i>) </td>
<td> Returns a new DStream by applying func (a RDD-to-RDD function) to every RDD of the stream. This can be used to do arbitrary RDD operations on the DStream. </td> <td> Returns a new DStream by applying func (a RDD-to-RDD function) to every RDD of the stream. This can be used to do arbitrary RDD operations on the DStream. </td>
</tr> </tr>
<tr>
<td> <b>updateStateByKey</b>(<i>func</i>) </td>
<td> Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key. This can be used to track session state by using the session-id as the key and updating the session state as new data is received.</td>
</tr>
</table> </table>
Spark Streaming features windowed computations, which allow you to report statistics over a sliding window of data. All window functions take a <i>windowDuration</i>, which represents the width of the window and a <i>slideTime</i>, which represents the frequency during which the window is calculated. Spark Streaming features windowed computations, which allow you to apply transformations over a sliding window of data. All window functions take a <i>windowDuration</i>, which represents the width of the window and a <i>slideTime</i>, which represents the frequency during which the window is calculated.
<table class="table"> <table class="table">
<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr> <tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
<tr> <tr>
<td> <b>window</b>(<i>windowDuration</i>, </i>slideTime</i>) </td> <td> <b>window</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
<td> Return a new stream which is computed based on windowed batches of the source stream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval. <td> Return a new DStream which is computed based on windowed batches of the source DStream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
</td> </td>
</tr> </tr>
<tr> <tr>
<td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideTime</i>) </td> <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
<td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>. <td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td> </td>
</tr> </tr>
<tr> <tr>
<td> <b>reduceByWindow</b>(<i>func</i>, <i>windowDuration</i>, </i>slideDuration</i>) </td> <td> <b>reduceByWindow</b>(<i>func</i>, <i>windowDuration</i>, <i>slideDuration</i>) </td>
<td> Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using <i>func</i>. The function should be associative so that it can be computed correctly in parallel. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>. <td> Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using <i>func</i>. The function should be associative so that it can be computed correctly in parallel. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td> </td>
</tr> </tr>
<tr> <tr>
<td> <b>groupByKeyAndWindow</b>(windowDuration, slideDuration, [<i>numTasks</i>]) <td> <b>groupByKeyAndWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>])
</td> </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs over a sliding window. <br /> <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together values of each key over batches in a sliding window. <br />
<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>. <b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.</td>
</tr>
<tr>
<td> <b>reduceByKeyAndWindow</b>(<i>func</i>, <i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>]) </td>
<td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function <i>func</i> over batches in a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
<tr>
<td> <b>reduceByKeyAndWindow</b>(<i>func</i>, <i>invFunc</i>, <i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>]) </td>
<td> A more efficient version of the above <code>reduceByKeyAndWindow()</code> where the reduce value of each window is calculated
incrementally using the reduce values of the previous window. This is done by reducing the new data that enter the sliding window, and "inverse reducing" the old data that leave the window. An example would be that of "adding" and "subtracting" counts of keys as the window slides. However, it is applicable to only "invertible reduce functions", that is, those reduce functions which have a corresponding "inverse reduce" function (taken as parameter <i>invFunc</i>. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td> </td>
</tr> </tr>
<tr> <tr>
<td> <b>reduceByKeyAndWindow</b>(<i>func</i>, [<i>numTasks</i>]) </td> <td> <b>countByValueAndWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function over batches within a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument. <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Long) pairs where the value of each key is its frequency within a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td> </td>
</tr>
<tr>
<td> <b>countByKeyAndWindow</b>([<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Int) pairs where the values for each key are the count within a sliding window. Like in <code>countByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr> </tr>
</table> </table>
@ -147,7 +162,7 @@ A complete list of DStream operations is available in the API documentation of [
When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined: When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined:
<table class="table"> <table class="table">
<tr><th style="width:25%">Operator</th><th>Meaning</th></tr> <tr><th style="width:30%">Operator</th><th>Meaning</th></tr>
<tr> <tr>
<td> <b>foreach</b>(<i>func</i>) </td> <td> <b>foreach</b>(<i>func</i>) </td>
<td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td> <td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td>
@ -176,11 +191,6 @@ When an output operator is called, it triggers the computation of a stream. Curr
</table> </table>
## 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 DStream 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()`.
Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence).
# Starting the Streaming computation # Starting the Streaming computation
All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using
{% highlight scala %} {% highlight scala %}
@ -192,8 +202,8 @@ Conversely, the computation can be stopped by using
ssc.stop() ssc.stop()
{% endhighlight %} {% endhighlight %}
# Example - NetworkWordCount.scala # Example
A good example to start off is the spark.streaming.examples.NetworkWordCount. This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in <Spark repo>/streaming/src/main/scala/spark/streaming/examples/WordCountNetwork.scala. A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` .
{% highlight scala %} {% highlight scala %}
import spark.streaming.{Seconds, StreamingContext} import spark.streaming.{Seconds, StreamingContext}
@ -202,7 +212,7 @@ import spark.streaming.StreamingContext._
// Create the context and set up a network input stream to receive from a host:port // Create the context and set up a network input stream to receive from a host:port
val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1)) val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1))
val lines = ssc.networkTextStream(args(1), args(2).toInt) val lines = ssc.socketTextStream(args(1), args(2).toInt)
// Split the lines into words, count them, and print some of the counts on the master // Split the lines into words, count them, and print some of the counts on the master
val words = lines.flatMap(_.split(" ")) val words = lines.flatMap(_.split(" "))
@ -213,6 +223,8 @@ wordCounts.print()
ssc.start() ssc.start()
{% endhighlight %} {% endhighlight %}
The `socketTextStream` returns a DStream of lines received from a TCP socket-based source. The `lines` DStream is _transformed_ into a DStream using the `flatMap` operation, where each line is split into words. This `words` DStream is then mapped to a DStream of `(word, 1)` pairs, which is finally reduced to get the word counts. `wordCounts.print()` will print 10 of the counts generated every second.
To run this example on your local machine, you need to first run a Netcat server by using To run this example on your local machine, you need to first run a Netcat server by using
{% highlight bash %} {% highlight bash %}
@ -260,6 +272,33 @@ Time: 1357008430000 ms
</td> </td>
</table> </table>
You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run 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()`.
For input streams that receive data from the network (that is, subclasses of NetworkInputDStream like FlumeInputDStream and KafkaInputDStream), the default persistence level is set to replicate the data to two nodes for fault-tolerance.
Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence).
# RDD Checkpointing within DStreams
A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation.
Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this metadata, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try.
To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using
{% highlight scala %}
ssc.checkpoint(hdfsPath) // assuming ssc is the StreamingContext
{% endhighlight %}
The interval of checkpointing of a DStream can be set by using
{% highlight scala %}
dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple of slide duration of dstream
{% endhighlight %}
For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds.
# Performance Tuning # Performance Tuning
@ -273,17 +312,21 @@ Getting the best performance of a Spark Streaming application on a cluster requi
There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
### Level of Parallelism ### Level of Parallelism
Cluster resources maybe underutilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
### Data Serialization ### Data Serialization
The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it. The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
* Serialization of RDD data in Spark: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC.
* Serialization of input data: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck. * **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC.
* **Serialization of input data**: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck.
### Task Launching Overheads ### Task Launching Overheads
If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes:
* Task Serialization: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves.
* Execution mode: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details. * **Task Serialization**: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves.
* **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details.
These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable.
## Setting the Right Batch Size ## Setting the Right Batch Size
@ -292,22 +335,183 @@ For a Spark Streaming application running on a cluster to be stable, the process
A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
## 24/7 Operation ## 24/7 Operation
By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.delay` to the number of minutes you want any metadata to persist. For example, setting `spark.cleaner.delay` to 10 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so. This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
## Memory Tuning ## Memory Tuning
Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times. Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times.
* <b>Default persistence level of DStreams</b>: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses. * **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses.
* <b>Concurrent garbage collector</b>: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. * **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times.
# Master Fault-tolerance (Alpha) # Fault-tolerance Properties
TODO In this section, we are going to discuss the behavior of Spark Streaming application in the event of a node failure. To understand this, let us remember the basic fault-tolerance properties of Spark's RDDs.
* Checkpointing of DStream graph 1. An RDD is an immutable, and deterministically re-computable, distributed dataset. Each RDD remembers the lineage of deterministic operations that were used on a fault-tolerant input dataset to create it.
1. If any partition of an RDD is lost due to a worker node failure, then that partition can be re-computed from the original fault-tolerant dataset using the lineage of operations.
* Recovery from master faults Since all data transformations in Spark Streaming are based on RDD operations, as long as the input dataset is present, all intermediate data can recomputed. Keeping these properties in mind, we are going to discuss the failure semantics in more detail.
* Current state and future directions ## Failure of a Worker Node
There are two failure behaviors based on which input sources are used.
1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
1. _Using any input source that receives data through a network_ - For network-based data sources like Kafka and Flume, the received input data is replicated in memory between nodes of the cluster (default replication factor is 2). So if a worker node fails, then the system can recompute the lost from the the left over copy of the input data. However, if the worker node where a network receiver was running fails, then a tiny bit of data may be lost, that is, the data received by the system but not yet replicated to other node(s). The receiver will be started on a different node and it will continue to receive data.
Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations.
## Failure of the Driver Node
A system that is required to operate 24/7 needs to be able tolerate the failure of the driver node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. This checkpointing is enabled by setting a HDFS directory for checkpointing using `ssc.checkpoint(<checkpoint directory>)` as described [earlier](#rdd-checkpointing-within-dstreams). To elaborate, the following state is periodically saved to a file.
1. The DStream operator graph (input streams, output streams, etc.)
1. The configuration of each DStream (checkpoint interval, etc.)
1. The RDD checkpoint files of each DStream
All this is periodically saved in the file `<checkpoint directory>/graph`. To recover, a new Streaming Context can be created with this directory by using
{% highlight scala %}
val ssc = new StreamingContext(checkpointDirectory)
{% endhighlight %}
On calling `ssc.start()` on this new context, the following steps are taken by the system
1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it last checkpointed. This is also done for those time steps that were previously scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc.
1. Restart the network receivers, if any, and continue receiving new data.
In the current _alpha_ release, there are two different failure behaviors based on which input sources are used.
1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
1. _Using any input source that receives data through a network_ - The received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely.
In future releases, we will support full recoverability for all input sources. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data.
To better understand the behavior of the system under driver failure with a HDFS source, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure.
<table class="table">
<!-- Results table headers -->
<tr>
<th> Time </th>
<th> Number of lines in input file </th>
<th> Output without driver failure </th>
<th> Output with driver failure </th>
</tr>
<tr>
<td>1</td>
<td>10</td>
<td>10</td>
<td>10</td>
</tr>
<tr>
<td>2</td>
<td>20</td>
<td>20</td>
<td>20</td>
</tr>
<tr>
<td>3</td>
<td>30</td>
<td>30</td>
<td>30</td>
</tr>
<tr>
<td>4</td>
<td>40</td>
<td>40</td>
<td>[DRIVER FAILS]<br />no output</td>
</tr>
<tr>
<td>5</td>
<td>50</td>
<td>50</td>
<td>no output</td>
</tr>
<tr>
<td>6</td>
<td>60</td>
<td>60</td>
<td>no output</td>
</tr>
<tr>
<td>7</td>
<td>70</td>
<td>70</td>
<td>[DRIVER RECOVERS]<br />40, 50, 60, 70</td>
</tr>
<tr>
<td>8</td>
<td>80</td>
<td>80</td>
<td>80</td>
</tr>
<tr>
<td>9</td>
<td>90</td>
<td>90</td>
<td>90</td>
</tr>
<tr>
<td>10</td>
<td>100</td>
<td>100</td>
<td>100</td>
</tr>
</table>
If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery.
# Java API
Similar to [Spark's Java API](java-programming-guide.html), we also provide a Java API for Spark Streaming which allows all its features to be accessible from a Java program. This is defined in [spark.streaming.api.java] (api/streaming/index.html#spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#spark.streaming.api.java.JavaDStream) classes that provide the same methods as their Scala counterparts, but take Java functions (that is, Function, and Function2) and return Java data and collection types. Some of the key points to note are:
1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#spark.api.java.function.Function) and [Function2](api/core/index.html#spark.api.java.function.Function2)
1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types.
Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java`
The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart.
{% highlight java %}
JavaStreamingContext ssc = new JavaStreamingContext(mesosUrl, "NetworkWordCount", Seconds(1));
JavaDStream<String> lines = ssc.socketTextStream(ip, port);
{% endhighlight %}
Then the `lines` are split into words by using the `flatMap` function and [FlatMapFunction](api/core/index.html#spark.api.java.function.FlatMapFunction).
{% highlight java %}
JavaDStream<String> words = lines.flatMap(
new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(x.split(" "));
}
});
{% endhighlight %}
The `words` is then mapped to a [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#spark.api.java.function.PairFunction). This is reduced by using `reduceByKey` and [Function2](api/core/index.html#spark.api.java.function.Function2).
{% highlight java %}
JavaPairDStream<String, Integer> wordCounts = words.map(
new PairFunction<String, String, Integer>() {
@Override
public Tuple2<String, Integer> call(String s) throws Exception {
return new Tuple2<String, Integer>(s, 1);
}
}).reduceByKey(
new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer i1, Integer i2) throws Exception {
return i1 + i2;
}
});
{% endhighlight %}
# Where to Go from Here
* API docs - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package)
* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples)
* [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)

View file

@ -213,10 +213,10 @@ but at a high level, managing how frequently full GC takes place can help in red
Clusters will not be fully utilized unless you set the level of parallelism for each operation high Clusters will not be fully utilized unless you set the level of parallelism for each operation high
enough. Spark automatically sets the number of "map" tasks to run on each file according to its size enough. Spark automatically sets the number of "map" tasks to run on each file according to its size
(though you can control it through optional parameters to `SparkContext.textFile`, etc), but for (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for
distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses a default value of 8. distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest
You can pass the level of parallelism as a second argument (see the parent RDD's number of partitions. You can pass the level of parallelism as a second argument
[`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation), (see the [`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation),
or set the system property `spark.default.parallelism` to change the default. or set the system property `spark.default.parallelism` to change the default.
In general, we recommend 2-3 tasks per CPU core in your cluster. In general, we recommend 2-3 tasks per CPU core in your cluster.

View file

@ -1,4 +1,4 @@
This folder contains a script, spark-ec2, for launching Spark clusters on This folder contains a script, spark-ec2, for launching Spark clusters on
Amazon EC2. Usage instructions are available online at: Amazon EC2. Usage instructions are available online at:
https://github.com/mesos/spark/wiki/Running-Spark-on-Amazon-EC2 http://spark-project.org/docs/latest/ec2-scripts.html

View file

@ -35,7 +35,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType
from boto import ec2 from boto import ec2
# A static URL from which to figure out the latest Mesos EC2 AMI # A static URL from which to figure out the latest Mesos EC2 AMI
LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.6" LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.7"
# Configure and parse our command-line arguments # Configure and parse our command-line arguments
@ -83,16 +83,16 @@ def parse_args():
help="If specified, launch slaves as spot instances with the given " + help="If specified, launch slaves as spot instances with the given " +
"maximum price (in dollars)") "maximum price (in dollars)")
parser.add_option("--cluster-type", type="choice", metavar="TYPE", parser.add_option("--cluster-type", type="choice", metavar="TYPE",
choices=["mesos", "standalone"], default="mesos", choices=["mesos", "standalone"], default="standalone",
help="'mesos' for a Mesos cluster, 'standalone' for a standalone " + help="'mesos' for a Mesos cluster, 'standalone' for a standalone " +
"Spark cluster (default: mesos)") "Spark cluster (default: standalone)")
parser.add_option("--ganglia", action="store_true", default=True, parser.add_option("--ganglia", action="store_true", default=True,
help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + help="Setup Ganglia monitoring on cluster (default: on). NOTE: " +
"the Ganglia page will be publicly accessible") "the Ganglia page will be publicly accessible")
parser.add_option("--no-ganglia", action="store_false", dest="ganglia", parser.add_option("--no-ganglia", action="store_false", dest="ganglia",
help="Disable Ganglia monitoring for the cluster") help="Disable Ganglia monitoring for the cluster")
parser.add_option("--new-scripts", action="store_true", default=False, parser.add_option("--old-scripts", action="store_true", default=False,
help="Use new spark-ec2 scripts, for Spark >= 0.7 AMIs") help="Use old mesos-ec2 scripts, for Spark <= 0.6 AMIs")
parser.add_option("-u", "--user", default="root", parser.add_option("-u", "--user", default="root",
help="The SSH user you want to connect as (default: root)") help="The SSH user you want to connect as (default: root)")
parser.add_option("--delete-groups", action="store_true", default=False, parser.add_option("--delete-groups", action="store_true", default=False,
@ -383,7 +383,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k
if opts.ganglia: if opts.ganglia:
modules.append('ganglia') modules.append('ganglia')
if opts.new_scripts: if not opts.old_scripts:
# NOTE: We should clone the repository before running deploy_files to # NOTE: We should clone the repository before running deploy_files to
# prevent ec2-variables.sh from being overwritten # prevent ec2-variables.sh from being overwritten
ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git") ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git")
@ -393,7 +393,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k
zoo_nodes, modules) zoo_nodes, modules)
print "Running setup on master..." print "Running setup on master..."
if not opts.new_scripts: if opts.old_scripts:
if opts.cluster_type == "mesos": if opts.cluster_type == "mesos":
setup_mesos_cluster(master, opts) setup_mesos_cluster(master, opts)
elif opts.cluster_type == "standalone": elif opts.cluster_type == "standalone":

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>
@ -20,11 +20,10 @@
<artifactId>jetty-server</artifactId> <artifactId>jetty-server</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.twitter4j</groupId> <groupId>com.twitter</groupId>
<artifactId>twitter4j-stream</artifactId> <artifactId>algebird-core_2.9.2</artifactId>
<version>3.0.3</version> <version>0.1.8</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.scalatest</groupId> <groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.version}</artifactId> <artifactId>scalatest_${scala.version}</artifactId>

View file

@ -10,6 +10,9 @@ import java.util.Arrays;
import java.util.StringTokenizer; import java.util.StringTokenizer;
import java.util.Random; import java.util.Random;
/**
* Logistic regression based classification.
*/
public class JavaHdfsLR { public class JavaHdfsLR {
static int D = 10; // Number of dimensions static int D = 10; // Number of dimensions
@ -85,7 +88,8 @@ public class JavaHdfsLR {
System.exit(1); System.exit(1);
} }
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR"); JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR",
System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
JavaRDD<String> lines = sc.textFile(args[1]); JavaRDD<String> lines = sc.textFile(args[1]);
JavaRDD<DataPoint> points = lines.map(new ParsePoint()).cache(); JavaRDD<DataPoint> points = lines.map(new ParsePoint()).cache();
int ITERATIONS = Integer.parseInt(args[2]); int ITERATIONS = Integer.parseInt(args[2]);

View file

@ -28,7 +28,7 @@ public class JavaTC {
Tuple2<Integer, Integer> e = new Tuple2<Integer, Integer>(from, to); Tuple2<Integer, Integer> e = new Tuple2<Integer, Integer>(from, to);
if (from != to) edges.add(e); if (from != to) edges.add(e);
} }
return new ArrayList(edges); return new ArrayList<Tuple2<Integer, Integer>>(edges);
} }
static class ProjectFn extends PairFunction<Tuple2<Integer, Tuple2<Integer, Integer>>, static class ProjectFn extends PairFunction<Tuple2<Integer, Tuple2<Integer, Integer>>,
@ -46,7 +46,8 @@ public class JavaTC {
System.exit(1); System.exit(1);
} }
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC"); JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC",
System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2;
JavaPairRDD<Integer, Integer> tc = sc.parallelizePairs(generateGraph(), slices).cache(); JavaPairRDD<Integer, Integer> tc = sc.parallelizePairs(generateGraph(), slices).cache();

View file

@ -18,7 +18,8 @@ public class JavaWordCount {
System.exit(1); System.exit(1);
} }
JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount"); JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount",
System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
JavaRDD<String> lines = ctx.textFile(args[1], 1); JavaRDD<String> lines = ctx.textFile(args[1], 1);
JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() { JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@ -29,7 +30,7 @@ public class JavaWordCount {
JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() { JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() {
public Tuple2<String, Integer> call(String s) { public Tuple2<String, Integer> call(String s) {
return new Tuple2(s, 1); return new Tuple2<String, Integer>(s, 1);
} }
}); });

View file

@ -32,7 +32,8 @@ public class JavaFlumeEventCount {
Duration batchInterval = new Duration(2000); Duration batchInterval = new Duration(2000);
JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval); JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port); JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);

View file

@ -23,19 +23,19 @@ import spark.streaming.api.java.JavaStreamingContext;
*/ */
public class JavaNetworkWordCount { public class JavaNetworkWordCount {
public static void main(String[] args) { public static void main(String[] args) {
if (args.length < 2) { if (args.length < 3) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" + System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1"); "In local mode, <master> should be 'local[n]' with n > 1");
System.exit(1); System.exit(1);
} }
// Create the context with a 1 second batch size // Create the context with a 1 second batch size
JavaStreamingContext ssc = new JavaStreamingContext( JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
args[0], "NetworkWordCount", new Duration(1000)); new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
// Create a NetworkInputDStream on target ip:port and count the // Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc') // words in input stream of \n delimited test (eg. generated by 'nc')
JavaDStream<String> lines = ssc.networkTextStream(args[1], Integer.parseInt(args[2])); JavaDStream<String> lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2]));
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() { JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override @Override
public Iterable<String> call(String x) { public Iterable<String> call(String x) {

View file

@ -22,7 +22,8 @@ public class JavaQueueStream {
} }
// Create the context // Create the context
JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000)); JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000),
System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
// Create the queue through which RDDs can be pushed to // Create the queue through which RDDs can be pushed to
// a QueueInputDStream // a QueueInputDStream

View file

@ -9,19 +9,21 @@ object BroadcastTest {
System.exit(1) System.exit(1)
} }
val spark = new SparkContext(args(0), "Broadcast Test") val sc = new SparkContext(args(0), "Broadcast Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2 val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000 val num = if (args.length > 2) args(2).toInt else 1000000
var arr1 = new Array[Int](num) var arr1 = new Array[Int](num)
for (i <- 0 until arr1.length) for (i <- 0 until arr1.length) {
arr1(i) = i arr1(i) = i
}
for (i <- 0 until 2) { for (i <- 0 until 2) {
println("Iteration " + i) println("Iteration " + i)
println("===========") println("===========")
val barr1 = spark.broadcast(arr1) val barr1 = sc.broadcast(arr1)
spark.parallelize(1 to 10, slices).foreach { sc.parallelize(1 to 10, slices).foreach {
i => println(barr1.value.size) i => println(barr1.value.size)
} }
} }

View file

@ -9,9 +9,10 @@ object ExceptionHandlingTest {
System.exit(1) System.exit(1)
} }
val sc = new SparkContext(args(0), "ExceptionHandlingTest") val sc = new SparkContext(args(0), "ExceptionHandlingTest",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
sc.parallelize(0 until sc.defaultParallelism).foreach { i => sc.parallelize(0 until sc.defaultParallelism).foreach { i =>
if (Math.random > 0.75) if (math.random > 0.75)
throw new Exception("Testing exception handling") throw new Exception("Testing exception handling")
} }

View file

@ -9,14 +9,15 @@ object GroupByTest {
if (args.length == 0) { if (args.length == 0) {
System.err.println("Usage: GroupByTest <master> [numMappers] [numKVPairs] [KeySize] [numReducers]") System.err.println("Usage: GroupByTest <master> [numMappers] [numKVPairs] [KeySize] [numReducers]")
System.exit(1) System.exit(1)
} }
var numMappers = if (args.length > 1) args(1).toInt else 2 var numMappers = if (args.length > 1) args(1).toInt else 2
var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var numKVPairs = if (args.length > 2) args(2).toInt else 1000
var valSize = if (args.length > 3) args(3).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000
var numReducers = if (args.length > 4) args(4).toInt else numMappers var numReducers = if (args.length > 4) args(4).toInt else numMappers
val sc = new SparkContext(args(0), "GroupBy Test") val sc = new SparkContext(args(0), "GroupBy Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random val ranGen = new Random

View file

@ -4,7 +4,8 @@ import spark._
object HdfsTest { object HdfsTest {
def main(args: Array[String]) { def main(args: Array[String]) {
val sc = new SparkContext(args(0), "HdfsTest") val sc = new SparkContext(args(0), "HdfsTest",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val file = sc.textFile(args(1)) val file = sc.textFile(args(1))
val mapped = file.map(s => s.length).cache() val mapped = file.map(s => s.length).cache()
for (iter <- 1 to 10) { for (iter <- 1 to 10) {

View file

@ -1,11 +1,13 @@
package spark.examples package spark.examples
import java.util.Random
import scala.math.sqrt import scala.math.sqrt
import cern.jet.math._ import cern.jet.math._
import cern.colt.matrix._ import cern.colt.matrix._
import cern.colt.matrix.linalg._ import cern.colt.matrix.linalg._
/**
* Alternating least squares matrix factorization.
*/
object LocalALS { object LocalALS {
// Parameters set through command line arguments // Parameters set through command line arguments
var M = 0 // Number of movies var M = 0 // Number of movies

View file

@ -6,6 +6,9 @@ import spark.SparkContext._
import scala.collection.mutable.HashMap import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet import scala.collection.mutable.HashSet
/**
* K-means clustering.
*/
object LocalKMeans { object LocalKMeans {
val N = 1000 val N = 1000
val R = 1000 // Scaling factor val R = 1000 // Scaling factor

View file

@ -3,6 +3,9 @@ package spark.examples
import java.util.Random import java.util.Random
import spark.util.Vector import spark.util.Vector
/**
* Logistic regression based classification.
*/
object LocalLR { object LocalLR {
val N = 10000 // Number of data points val N = 10000 // Number of data points
val D = 10 // Number of dimensions val D = 10 // Number of dimensions

View file

@ -26,7 +26,9 @@ object LogQuery {
System.err.println("Usage: LogQuery <master> [logFile]") System.err.println("Usage: LogQuery <master> [logFile]")
System.exit(1) System.exit(1)
} }
val sc = new SparkContext(args(0), "Log Query")
val sc = new SparkContext(args(0), "Log Query",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val dataSet = val dataSet =
if (args.length == 2) sc.textFile(args(1)) if (args.length == 2) sc.textFile(args(1))

View file

@ -9,21 +9,25 @@ object MultiBroadcastTest {
System.exit(1) System.exit(1)
} }
val spark = new SparkContext(args(0), "Broadcast Test") val sc = new SparkContext(args(0), "Broadcast Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2 val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000 val num = if (args.length > 2) args(2).toInt else 1000000
var arr1 = new Array[Int](num) var arr1 = new Array[Int](num)
for (i <- 0 until arr1.length) for (i <- 0 until arr1.length) {
arr1(i) = i arr1(i) = i
}
var arr2 = new Array[Int](num) var arr2 = new Array[Int](num)
for (i <- 0 until arr2.length) for (i <- 0 until arr2.length) {
arr2(i) = i arr2(i) = i
}
val barr1 = spark.broadcast(arr1) val barr1 = sc.broadcast(arr1)
val barr2 = spark.broadcast(arr2) val barr2 = sc.broadcast(arr2)
spark.parallelize(1 to 10, slices).foreach { sc.parallelize(1 to 10, slices).foreach {
i => println(barr1.value.size + barr2.value.size) i => println(barr1.value.size + barr2.value.size)
} }

View file

@ -18,7 +18,8 @@ object SimpleSkewedGroupByTest {
var numReducers = if (args.length > 4) args(4).toInt else numMappers var numReducers = if (args.length > 4) args(4).toInt else numMappers
var ratio = if (args.length > 5) args(5).toInt else 5.0 var ratio = if (args.length > 5) args(5).toInt else 5.0
val sc = new SparkContext(args(0), "GroupBy Test") val sc = new SparkContext(args(0), "GroupBy Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random val ranGen = new Random

View file

@ -16,13 +16,14 @@ object SkewedGroupByTest {
var valSize = if (args.length > 3) args(3).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000
var numReducers = if (args.length > 4) args(4).toInt else numMappers var numReducers = if (args.length > 4) args(4).toInt else numMappers
val sc = new SparkContext(args(0), "GroupBy Test") val sc = new SparkContext(args(0), "GroupBy Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random val ranGen = new Random
// map output sizes lineraly increase from the 1st to the last // map output sizes lineraly increase from the 1st to the last
numKVPairs = (1. * (p + 1) / numMappers * numKVPairs).toInt numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt
var arr1 = new Array[(Int, Array[Byte])](numKVPairs) var arr1 = new Array[(Int, Array[Byte])](numKVPairs)
for (i <- 0 until numKVPairs) { for (i <- 0 until numKVPairs) {
@ -31,11 +32,11 @@ object SkewedGroupByTest {
arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr) arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr)
} }
arr1 arr1
}.cache }.cache()
// Enforce that everything has been calculated and in cache // Enforce that everything has been calculated and in cache
pairs1.count pairs1.count()
println(pairs1.groupByKey(numReducers).count) println(pairs1.groupByKey(numReducers).count())
System.exit(0) System.exit(0)
} }

View file

@ -1,14 +1,14 @@
package spark.examples package spark.examples
import java.io.Serializable
import java.util.Random
import scala.math.sqrt import scala.math.sqrt
import cern.jet.math._ import cern.jet.math._
import cern.colt.matrix._ import cern.colt.matrix._
import cern.colt.matrix.linalg._ import cern.colt.matrix.linalg._
import spark._ import spark._
import scala.Option
/**
* Alternating least squares matrix factorization.
*/
object SparkALS { object SparkALS {
// Parameters set through command line arguments // Parameters set through command line arguments
var M = 0 // Number of movies var M = 0 // Number of movies
@ -70,30 +70,32 @@ object SparkALS {
} }
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length == 0) {
System.err.println("Usage: SparkALS <master> [<M> <U> <F> <iters> <slices>]")
System.exit(1)
}
var host = "" var host = ""
var slices = 0 var slices = 0
(0 to 5).map(i => { val options = (0 to 5).map(i => if (i < args.length) Some(args(i)) else None)
i match {
case a if a < args.length => Some(args(a)) options.toArray match {
case _ => None case Array(host_, m, u, f, iters, slices_) =>
} host = host_.get
}).toArray match { M = m.getOrElse("100").toInt
case Array(host_, m, u, f, iters, slices_) => { U = u.getOrElse("500").toInt
host = host_ getOrElse "local" F = f.getOrElse("10").toInt
M = (m getOrElse "100").toInt ITERATIONS = iters.getOrElse("5").toInt
U = (u getOrElse "500").toInt slices = slices_.getOrElse("2").toInt
F = (f getOrElse "10").toInt case _ =>
ITERATIONS = (iters getOrElse "5").toInt System.err.println("Usage: SparkALS <master> [<M> <U> <F> <iters> <slices>]")
slices = (slices_ getOrElse "2").toInt
}
case _ => {
System.err.println("Usage: SparkALS [<master> <M> <U> <F> <iters> <slices>]")
System.exit(1) System.exit(1)
}
} }
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
val spark = new SparkContext(host, "SparkALS")
val sc = new SparkContext(host, "SparkALS",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val R = generateR() val R = generateR()
@ -102,19 +104,19 @@ object SparkALS {
var us = Array.fill(U)(factory1D.random(F)) var us = Array.fill(U)(factory1D.random(F))
// Iteratively update movies then users // Iteratively update movies then users
val Rc = spark.broadcast(R) val Rc = sc.broadcast(R)
var msc = spark.broadcast(ms) var msb = sc.broadcast(ms)
var usc = spark.broadcast(us) var usb = sc.broadcast(us)
for (iter <- 1 to ITERATIONS) { for (iter <- 1 to ITERATIONS) {
println("Iteration " + iter + ":") println("Iteration " + iter + ":")
ms = spark.parallelize(0 until M, slices) ms = sc.parallelize(0 until M, slices)
.map(i => update(i, msc.value(i), usc.value, Rc.value)) .map(i => update(i, msb.value(i), usb.value, Rc.value))
.toArray .toArray
msc = spark.broadcast(ms) // Re-broadcast ms because it was updated msb = sc.broadcast(ms) // Re-broadcast ms because it was updated
us = spark.parallelize(0 until U, slices) us = sc.parallelize(0 until U, slices)
.map(i => update(i, usc.value(i), msc.value, algebra.transpose(Rc.value))) .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value)))
.toArray .toArray
usc = spark.broadcast(us) // Re-broadcast us because it was updated usb = sc.broadcast(us) // Re-broadcast us because it was updated
println("RMSE = " + rmse(R, ms, us)) println("RMSE = " + rmse(R, ms, us))
println() println()
} }

View file

@ -5,6 +5,9 @@ import scala.math.exp
import spark.util.Vector import spark.util.Vector
import spark._ import spark._
/**
* Logistic regression based classification.
*/
object SparkHdfsLR { object SparkHdfsLR {
val D = 10 // Numer of dimensions val D = 10 // Numer of dimensions
val rand = new Random(42) val rand = new Random(42)
@ -29,7 +32,8 @@ object SparkHdfsLR {
System.err.println("Usage: SparkHdfsLR <master> <file> <iters>") System.err.println("Usage: SparkHdfsLR <master> <file> <iters>")
System.exit(1) System.exit(1)
} }
val sc = new SparkContext(args(0), "SparkHdfsLR") val sc = new SparkContext(args(0), "SparkHdfsLR",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val lines = sc.textFile(args(1)) val lines = sc.textFile(args(1))
val points = lines.map(parsePoint _).cache() val points = lines.map(parsePoint _).cache()
val ITERATIONS = args(2).toInt val ITERATIONS = args(2).toInt

View file

@ -7,6 +7,9 @@ import spark.SparkContext._
import scala.collection.mutable.HashMap import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet import scala.collection.mutable.HashSet
/**
* K-means clustering.
*/
object SparkKMeans { object SparkKMeans {
val R = 1000 // Scaling factor val R = 1000 // Scaling factor
val rand = new Random(42) val rand = new Random(42)
@ -36,7 +39,8 @@ object SparkKMeans {
System.err.println("Usage: SparkLocalKMeans <master> <file> <k> <convergeDist>") System.err.println("Usage: SparkLocalKMeans <master> <file> <k> <convergeDist>")
System.exit(1) System.exit(1)
} }
val sc = new SparkContext(args(0), "SparkLocalKMeans") val sc = new SparkContext(args(0), "SparkLocalKMeans",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val lines = sc.textFile(args(1)) val lines = sc.textFile(args(1))
val data = lines.map(parseVector _).cache() val data = lines.map(parseVector _).cache()
val K = args(2).toInt val K = args(2).toInt

View file

@ -5,6 +5,9 @@ import scala.math.exp
import spark.util.Vector import spark.util.Vector
import spark._ import spark._
/**
* Logistic regression based classification.
*/
object SparkLR { object SparkLR {
val N = 10000 // Number of data points val N = 10000 // Number of data points
val D = 10 // Numer of dimensions val D = 10 // Numer of dimensions
@ -28,7 +31,8 @@ object SparkLR {
System.err.println("Usage: SparkLR <master> [<slices>]") System.err.println("Usage: SparkLR <master> [<slices>]")
System.exit(1) System.exit(1)
} }
val sc = new SparkContext(args(0), "SparkLR") val sc = new SparkContext(args(0), "SparkLR",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val numSlices = if (args.length > 1) args(1).toInt else 2 val numSlices = if (args.length > 1) args(1).toInt else 2
val points = sc.parallelize(generateData, numSlices).cache() val points = sc.parallelize(generateData, numSlices).cache()

View file

@ -10,7 +10,8 @@ object SparkPi {
System.err.println("Usage: SparkPi <master> [<slices>]") System.err.println("Usage: SparkPi <master> [<slices>]")
System.exit(1) System.exit(1)
} }
val spark = new SparkContext(args(0), "SparkPi") val spark = new SparkContext(args(0), "SparkPi",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2 val slices = if (args.length > 1) args(1).toInt else 2
val n = 100000 * slices val n = 100000 * slices
val count = spark.parallelize(1 to n, slices).map { i => val count = spark.parallelize(1 to n, slices).map { i =>

View file

@ -9,7 +9,6 @@ import scala.collection.mutable
* Transitive closure on a graph. * Transitive closure on a graph.
*/ */
object SparkTC { object SparkTC {
val numEdges = 200 val numEdges = 200
val numVertices = 100 val numVertices = 100
val rand = new Random(42) val rand = new Random(42)
@ -29,7 +28,8 @@ object SparkTC {
System.err.println("Usage: SparkTC <master> [<slices>]") System.err.println("Usage: SparkTC <master> [<slices>]")
System.exit(1) System.exit(1)
} }
val spark = new SparkContext(args(0), "SparkTC") val spark = new SparkContext(args(0), "SparkTC",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2 val slices = if (args.length > 1) args(1).toInt else 2
var tc = spark.parallelize(generateGraph, slices).cache() var tc = spark.parallelize(generateGraph, slices).cache()

View file

@ -0,0 +1,158 @@
package spark.streaming.examples
import scala.collection.mutable.LinkedList
import scala.util.Random
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.actorRef2Scala
import spark.streaming.Seconds
import spark.streaming.StreamingContext
import spark.streaming.StreamingContext.toPairDStreamFunctions
import spark.streaming.receivers.Receiver
import spark.util.AkkaUtils
case class SubscribeReceiver(receiverActor: ActorRef)
case class UnsubscribeReceiver(receiverActor: ActorRef)
/**
* Sends the random content to every receiver subscribed with 1/2
* second delay.
*/
class FeederActor extends Actor {
val rand = new Random()
var receivers: LinkedList[ActorRef] = new LinkedList[ActorRef]()
val strings: Array[String] = Array("words ", "may ", "count ")
def makeMessage(): String = {
val x = rand.nextInt(3)
strings(x) + strings(2 - x)
}
/*
* A thread to generate random messages
*/
new Thread() {
override def run() {
while (true) {
Thread.sleep(500)
receivers.foreach(_ ! makeMessage)
}
}
}.start()
def receive: Receive = {
case SubscribeReceiver(receiverActor: ActorRef) =>
println("received subscribe from %s".format(receiverActor.toString))
receivers = LinkedList(receiverActor) ++ receivers
case UnsubscribeReceiver(receiverActor: ActorRef) =>
println("received unsubscribe from %s".format(receiverActor.toString))
receivers = receivers.dropWhile(x => x eq receiverActor)
}
}
/**
* A sample actor as receiver, is also simplest. This receiver actor
* goes and subscribe to a typical publisher/feeder actor and receives
* data.
*
* @see [[spark.streaming.examples.FeederActor]]
*/
class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String)
extends Actor with Receiver {
lazy private val remotePublisher = context.actorFor(urlOfPublisher)
override def preStart = remotePublisher ! SubscribeReceiver(context.self)
def receive = {
case msg context.parent ! pushBlock(msg.asInstanceOf[T])
}
override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self)
}
/**
* A sample feeder actor
*
* Usage: FeederActor <hostname> <port>
* <hostname> and <port> describe the AkkaSystem that Spark Sample feeder would start on.
*/
object FeederActor {
def main(args: Array[String]) {
if(args.length < 2){
System.err.println(
"Usage: FeederActor <hostname> <port>\n"
)
System.exit(1)
}
val Seq(host, port) = args.toSeq
val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1
val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor")
println("Feeder started as:" + feeder)
actorSystem.awaitTermination();
}
}
/**
* A sample word count program demonstrating the use of plugging in
* Actor as Receiver
* Usage: ActorWordCount <master> <hostname> <port>
* <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
* <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
*
* To run this example locally, you may run Feeder Actor as
* `$ ./run spark.streaming.examples.FeederActor 127.0.1.1 9999`
* and then run the example
* `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
*/
object ActorWordCount {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println(
"Usage: ActorWordCount <master> <hostname> <port>" +
"In local mode, <master> should be 'local[n]' with n > 1")
System.exit(1)
}
val Seq(master, host, port) = args.toSeq
// 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")))
/*
* Following is the use of actorStream to plug in custom actor as receiver
*
* An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e type of data received and InputDstream
* should be same.
*
* For example: Both actorStream and SampleActorReceiver are parameterized
* to same type to ensure type safety.
*/
val lines = ssc.actorStream[String](
Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format(
host, port.toInt))), "SampleReceiver")
//compute wordcount
lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print()
ssc.start()
}
}

View file

@ -30,7 +30,8 @@ object FlumeEventCount {
val batchInterval = Milliseconds(2000) val batchInterval = Milliseconds(2000)
// Create the context and set the batch size // Create the context and set the batch size
val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval) val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval,
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Create a flume stream // Create a flume stream
val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)

View file

@ -22,7 +22,8 @@ object HdfsWordCount {
} }
// Create the context // Create the context
val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2)) val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Create the FileInputDStream on the directory and use the // Create the FileInputDStream on the directory and use the
// stream to count words in new files created // stream to count words in new files created

View file

@ -10,22 +10,34 @@ import spark.streaming.StreamingContext._
import spark.storage.StorageLevel import spark.storage.StorageLevel
import spark.streaming.util.RawTextHelper._ import spark.streaming.util.RawTextHelper._
/**
* Consumes messages from one or more topics in Kafka and does wordcount.
* Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>
* <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
* <zkQuorum> is a list of one or more zookeeper servers that make quorum
* <group> is the name of kafka consumer group
* <topics> is a list of one or more kafka topics to consume from
* <numThreads> is the number of threads the kafka consumer should use
*
* Example:
* `./run spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
*/
object KafkaWordCount { object KafkaWordCount {
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length < 6) { if (args.length < 5) {
System.err.println("Usage: KafkaWordCount <master> <hostname> <port> <group> <topics> <numThreads>") System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>")
System.exit(1) System.exit(1)
} }
val Array(master, hostname, port, group, topics, numThreads) = args val Array(master, zkQuorum, group, topics, numThreads) = args
val sc = new SparkContext(master, "KafkaWordCount") val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2),
val ssc = new StreamingContext(sc, Seconds(2)) System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
ssc.checkpoint("checkpoint") ssc.checkpoint("checkpoint")
val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
val lines = ssc.kafkaStream[String](hostname, port.toInt, group, topicpMap) val lines = ssc.kafkaStream[String](zkQuorum, group, topicpMap)
val words = lines.flatMap(_.split(" ")) val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
wordCounts.print() wordCounts.print()
@ -38,16 +50,16 @@ object KafkaWordCount {
object KafkaWordCountProducer { object KafkaWordCountProducer {
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length < 3) { if (args.length < 2) {
System.err.println("Usage: KafkaWordCountProducer <hostname> <port> <topic> <messagesPerSec> <wordsPerMessage>") System.err.println("Usage: KafkaWordCountProducer <zkQuorum> <topic> <messagesPerSec> <wordsPerMessage>")
System.exit(1) System.exit(1)
} }
val Array(hostname, port, topic, messagesPerSec, wordsPerMessage) = args val Array(zkQuorum, topic, messagesPerSec, wordsPerMessage) = args
// Zookeper connection properties // Zookeper connection properties
val props = new Properties() val props = new Properties()
props.put("zk.connect", hostname + ":" + port) props.put("zk.connect", zkQuorum)
props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("serializer.class", "kafka.serializer.StringEncoder")
val config = new ProducerConfig(props) val config = new ProducerConfig(props)

View file

@ -16,18 +16,19 @@ import spark.streaming.StreamingContext._
*/ */
object NetworkWordCount { object NetworkWordCount {
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length < 2) { if (args.length < 3) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" + System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1") "In local mode, <master> should be 'local[n]' with n > 1")
System.exit(1) System.exit(1)
} }
// Create the context with a 1 second batch size // Create the context with a 1 second batch size
val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1)) val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Create a NetworkInputDStream on target ip:port and count the // Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc') // words in input stream of \n delimited test (eg. generated by 'nc')
val lines = ssc.networkTextStream(args(1), args(2).toInt) val lines = ssc.socketTextStream(args(1), args(2).toInt)
val words = lines.flatMap(_.split(" ")) val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print() wordCounts.print()

View file

@ -15,7 +15,8 @@ object QueueStream {
} }
// Create the context // Create the context
val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1)) val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Create the queue through which RDDs can be pushed to // Create the queue through which RDDs can be pushed to
// a QueueInputDStream // a QueueInputDStream
@ -30,10 +31,10 @@ object QueueStream {
// Create and push some RDDs into // Create and push some RDDs into
for (i <- 1 to 30) { for (i <- 1 to 30) {
rddQueue += ssc.sc.makeRDD(1 to 1000, 10) rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10)
Thread.sleep(1000) Thread.sleep(1000)
} }
ssc.stop() ssc.stop()
System.exit(0) System.exit(0)
} }
} }

View file

@ -31,13 +31,14 @@ object RawNetworkGrep {
val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args
// Create the context // Create the context
val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis)) val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Warm up the JVMs on master and slave for JIT compilation to kick in // Warm up the JVMs on master and slave for JIT compilation to kick in
RawTextHelper.warmUp(ssc.sc) RawTextHelper.warmUp(ssc.sparkContext)
val rawStreams = (1 to numStreams).map(_ => val rawStreams = (1 to numStreams).map(_ =>
ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
val union = ssc.union(rawStreams) val union = ssc.union(rawStreams)
union.filter(_.contains("the")).count().foreach(r => union.filter(_.contains("the")).count().foreach(r =>
println("Grep count: " + r.collect().mkString)) println("Grep count: " + r.collect().mkString))

View file

@ -0,0 +1,94 @@
package spark.streaming.examples
import spark.streaming.{Seconds, StreamingContext}
import spark.storage.StorageLevel
import com.twitter.algebird._
import spark.streaming.StreamingContext._
import spark.SparkContext._
/**
* 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.
* <br>
* <strong>Note</strong> that since Algebird's implementation currently only supports Long inputs,
* the example operates on Long IDs. Once the implementation supports other inputs (such as String),
* the same approach could be used for computing popular topics for example.
* <p>
* <p>
* <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
* This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure
* 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
* percentage of the overall total count.
* <p><p>
* Algebird's implementation is a monoid, so we can succinctly merge two CMS instances in the reduce operation.
*/
object TwitterAlgebirdCMS {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: TwitterAlgebirdCMS <master> <twitter_username> <twitter_password>" +
" [filter1] [filter2] ... [filter n]")
System.exit(1)
}
// CMS parameters
val DELTA = 1E-3
val EPS = 0.01
val SEED = 1
val PERC = 0.001
// K highest frequency elements to take
val TOPK = 10
val Array(master, username, password) = args.slice(0, 3)
val filters = args.slice(3, args.length)
val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER)
val users = stream.map(status => status.getUser.getId)
val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC)
var globalCMS = cms.zero
val mm = new MapMonoid[Long, Int]()
var globalExact = Map[Long, Int]()
val approxTopUsers = users.mapPartitions(ids => {
ids.map(id => cms.create(id))
}).reduce(_ ++ _)
val exactTopUsers = users.map(id => (id, 1))
.reduceByKey((a, b) => a + b)
approxTopUsers.foreach(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
val partialTopK = partial.heavyHitters.map(id =>
(id, partial.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK)
globalCMS ++= partial
val globalTopK = globalCMS.heavyHitters.map(id =>
(id, globalCMS.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK)
println("Approx heavy hitters at %2.2f%% threshold this batch: %s".format(PERC,
partialTopK.mkString("[", ",", "]")))
println("Approx heavy hitters at %2.2f%% threshold overall: %s".format(PERC,
globalTopK.mkString("[", ",", "]")))
}
})
exactTopUsers.foreach(rdd => {
if (rdd.count() != 0) {
val partialMap = rdd.collect().toMap
val partialTopK = rdd.map(
{case (id, count) => (count, id)})
.sortByKey(ascending = false).take(TOPK)
globalExact = mm.plus(globalExact.toMap, partialMap)
val globalTopK = globalExact.toSeq.sortBy(_._2).reverse.slice(0, TOPK)
println("Exact heavy hitters this batch: %s".format(partialTopK.mkString("[", ",", "]")))
println("Exact heavy hitters overall: %s".format(globalTopK.mkString("[", ",", "]")))
}
})
ssc.start()
}
}

View file

@ -0,0 +1,72 @@
package spark.streaming.examples
import spark.streaming.{Seconds, StreamingContext}
import spark.storage.StorageLevel
import com.twitter.algebird.HyperLogLog._
import com.twitter.algebird.HyperLogLogMonoid
import spark.streaming.dstream.TwitterInputDStream
/**
* Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute
* a windowed and global estimate of the unique user IDs occurring in a Twitter stream.
* <p>
* <p>
* This <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
* blog post</a> and this
* <a href="http://highscalability.com/blog/2012/4/5/big-data-counting-how-to-count-a-billion-distinct-objects-us.html">blog post</a>
* have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for estimating
* the cardinality of a data stream, i.e. the number of unique elements.
* <p><p>
* Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the reduce operation.
*/
object TwitterAlgebirdHLL {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: TwitterAlgebirdHLL <master> <twitter_username> <twitter_password>" +
" [filter1] [filter2] ... [filter n]")
System.exit(1)
}
/** Bit size parameter for HyperLogLog, trades off accuracy vs size */
val BIT_SIZE = 12
val Array(master, username, password) = args.slice(0, 3)
val filters = args.slice(3, args.length)
val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER)
val users = stream.map(status => status.getUser.getId)
val hll = new HyperLogLogMonoid(BIT_SIZE)
var globalHll = hll.zero
var userSet: Set[Long] = Set()
val approxUsers = users.mapPartitions(ids => {
ids.map(id => hll(id))
}).reduce(_ + _)
val exactUsers = users.map(id => Set(id)).reduce(_ ++ _)
approxUsers.foreach(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
globalHll += partial
println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt))
println("Approx distinct users overall: %d".format(globalHll.estimatedSize.toInt))
}
})
exactUsers.foreach(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
userSet ++= partial
println("Exact distinct users this batch: %d".format(partial.size))
println("Exact distinct users overall: %d".format(userSet.size))
println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1) * 100))
}
})
ssc.start()
}
}

View file

@ -1,19 +1,19 @@
package spark.streaming.examples.twitter package spark.streaming.examples
import spark.streaming.StreamingContext._
import spark.streaming.{Seconds, StreamingContext} import spark.streaming.{Seconds, StreamingContext}
import StreamingContext._
import spark.SparkContext._ import spark.SparkContext._
import spark.storage.StorageLevel
/** /**
* Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
* stream. The stream is instantiated with credentials and optionally filters supplied by the * stream. The stream is instantiated with credentials and optionally filters supplied by the
* command line arguments. * command line arguments.
*
*/ */
object TwitterBasic { object TwitterPopularTags {
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length < 3) { if (args.length < 3) {
System.err.println("Usage: TwitterBasic <master> <twitter_username> <twitter_password>" + System.err.println("Usage: TwitterPopularTags <master> <twitter_username> <twitter_password>" +
" [filter1] [filter2] ... [filter n]") " [filter1] [filter2] ... [filter n]")
System.exit(1) System.exit(1)
} }
@ -21,10 +21,9 @@ object TwitterBasic {
val Array(master, username, password) = args.slice(0, 3) val Array(master, username, password) = args.slice(0, 3)
val filters = args.slice(3, args.length) val filters = args.slice(3, args.length)
val ssc = new StreamingContext(master, "TwitterBasic", Seconds(2)) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2),
val stream = new TwitterInputDStream(ssc, username, password, filters, System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
StorageLevel.MEMORY_ONLY_SER) val stream = ssc.twitterStream(username, password, filters)
ssc.registerInputStream(stream)
val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
@ -39,22 +38,17 @@ object TwitterBasic {
// Print popular hashtags // Print popular hashtags
topCounts60.foreach(rdd => { topCounts60.foreach(rdd => {
if (rdd.count() != 0) { val topList = rdd.take(5)
val topList = rdd.take(5) println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
}
}) })
topCounts10.foreach(rdd => { topCounts10.foreach(rdd => {
if (rdd.count() != 0) { val topList = rdd.take(5)
val topList = rdd.take(5) println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
}
}) })
ssc.start() ssc.start()
} }
} }

View file

@ -0,0 +1,74 @@
package spark.streaming.examples
import akka.actor.ActorSystem
import akka.actor.actorRef2Scala
import akka.zeromq._
import spark.streaming.{ Seconds, StreamingContext }
import spark.streaming.StreamingContext._
import akka.zeromq.Subscribe
/**
* A simple publisher for demonstration purposes, repeatedly publishes random Messages
* every one second.
*/
object SimpleZeroMQPublisher {
def main(args: Array[String]) = {
if (args.length < 2) {
System.err.println("Usage: SimpleZeroMQPublisher <zeroMQUrl> <topic> ")
System.exit(1)
}
val Seq(url, topic) = args.toSeq
val acs: ActorSystem = ActorSystem()
val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url))
val messages: Array[String] = Array("words ", "may ", "count ")
while (true) {
Thread.sleep(1000)
pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList)
}
acs.awaitTermination()
}
}
/**
* A sample wordcount with ZeroMQStream stream
*
* To work with zeroMQ, some native libraries have to be installed.
* Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide](http://www.zeromq.org/intro:get-the-software)
*
* Usage: ZeroMQWordCount <master> <zeroMQurl> <topic>
* In local mode, <master> should be 'local[n]' with n > 1
* <zeroMQurl> and <topic> describe where zeroMq publisher is running.
*
* To run this example locally, you may run publisher as
* `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
* and run the example as
* `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
*/
object ZeroMQWordCount {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println(
"Usage: ZeroMQWordCount <master> <zeroMQurl> <topic>" +
"In local mode, <master> should be 'local[n]' with n > 1")
System.exit(1)
}
val Seq(master, url, topic) = args.toSeq
// 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")))
def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator
//For this stream, a zeroMQ publisher should be running.
val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
}
}

View file

@ -24,20 +24,20 @@ object PageViewStream {
val port = args(2).toInt val port = args(2).toInt
// Create the context // Create the context
val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1)) val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
// Create a NetworkInputDStream on target host:port and convert each line to a PageView // Create a NetworkInputDStream on target host:port and convert each line to a PageView
val pageViews = ssc.networkTextStream(host, port) val pageViews = ssc.socketTextStream(host, port)
.flatMap(_.split("\n")) .flatMap(_.split("\n"))
.map(PageView.fromString(_)) .map(PageView.fromString(_))
// Return a count of views per URL seen in each batch // Return a count of views per URL seen in each batch
val pageCounts = pageViews.map(view => ((view.url, 1))).countByKey() val pageCounts = pageViews.map(view => view.url).countByValue()
// Return a sliding window of page views per URL in the last ten seconds // Return a sliding window of page views per URL in the last ten seconds
val slidingPageCounts = pageViews.map(view => ((view.url, 1))) val slidingPageCounts = pageViews.map(view => view.url)
.window(Seconds(10), Seconds(2)) .countByValueAndWindow(Seconds(10), Seconds(2))
.countByKey()
// Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds // Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds
@ -61,7 +61,7 @@ object PageViewStream {
.map("Unique active users: " + _) .map("Unique active users: " + _)
// An external dataset we want to join to this stream // An external dataset we want to join to this stream
val userList = ssc.sc.parallelize( val userList = ssc.sparkContext.parallelize(
Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq) Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq)
metric match { metric match {

10
pom.xml
View file

@ -3,7 +3,7 @@
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<packaging>pom</packaging> <packaging>pom</packaging>
<name>Spark Project Parent POM</name> <name>Spark Project Parent POM</name>
<url>http://spark-project.org/</url> <url>http://spark-project.org/</url>
@ -84,9 +84,9 @@
</snapshots> </snapshots>
</repository> </repository>
<repository> <repository>
<id>typesafe-repo</id> <id>akka-repo</id>
<name>Typesafe Repository</name> <name>Akka Repository</name>
<url>http://repo.typesafe.com/typesafe/releases/</url> <url>http://repo.akka.io/releases/</url>
<releases> <releases>
<enabled>true</enabled> <enabled>true</enabled>
</releases> </releases>
@ -514,7 +514,7 @@
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId> <artifactId>hadoop-core</artifactId>
<version>1.0.3</version> <version>1.0.4</version>
</dependency> </dependency>
</dependencies> </dependencies>
</dependencyManagement> </dependencyManagement>

View file

@ -9,8 +9,8 @@ import twirl.sbt.TwirlPlugin._
object SparkBuild extends Build { object SparkBuild extends Build {
// Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or
// "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop.
val HADOOP_VERSION = "1.0.3" val HADOOP_VERSION = "1.0.4"
val HADOOP_MAJOR_VERSION = "1" val HADOOP_MAJOR_VERSION = "1"
// For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2"
@ -35,7 +35,7 @@ object SparkBuild extends Build {
def sharedSettings = Defaults.defaultSettings ++ Seq( def sharedSettings = Defaults.defaultSettings ++ Seq(
organization := "org.spark-project", organization := "org.spark-project",
version := "0.7.0-SNAPSHOT", version := "0.7.1-SNAPSHOT",
scalaVersion := "2.9.2", scalaVersion := "2.9.2",
scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue
unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath },
@ -44,6 +44,9 @@ object SparkBuild extends Build {
transitiveClassifiers in Scope.GlobalScope := Seq("sources"), transitiveClassifiers in Scope.GlobalScope := Seq("sources"),
testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))),
// shared between both core and streaming.
resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
// For Sonatype publishing // For Sonatype publishing
resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
@ -114,7 +117,6 @@ object SparkBuild extends Build {
def coreSettings = sharedSettings ++ Seq( def coreSettings = sharedSettings ++ Seq(
name := "spark-core", name := "spark-core",
resolvers ++= Seq( resolvers ++= Seq(
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
"JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
"Spray Repository" at "http://repo.spray.cc/", "Spray Repository" at "http://repo.spray.cc/",
"Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/",
@ -155,9 +157,7 @@ object SparkBuild extends Build {
def examplesSettings = sharedSettings ++ Seq( def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples", name := "spark-examples",
libraryDependencies ++= Seq( libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8")
"org.twitter4j" % "twitter4j-stream" % "3.0.3"
)
) )
def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
@ -166,7 +166,9 @@ object SparkBuild extends Build {
name := "spark-streaming", name := "spark-streaming",
libraryDependencies ++= Seq( libraryDependencies ++= Seq(
"org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
"com.github.sgroschupf" % "zkclient" % "0.1" "com.github.sgroschupf" % "zkclient" % "0.1",
"org.twitter4j" % "twitter4j-stream" % "3.0.3",
"com.typesafe.akka" % "akka-zeromq" % "2.0.3"
) )
) ++ assemblySettings ++ extraAssemblySettings ) ++ assemblySettings ++ extraAssemblySettings

View file

@ -32,13 +32,13 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
""" """
def _do_python_join(rdd, other, numSplits, dispatch): def _do_python_join(rdd, other, numPartitions, dispatch):
vs = rdd.map(lambda (k, v): (k, (1, v))) vs = rdd.map(lambda (k, v): (k, (1, v)))
ws = other.map(lambda (k, v): (k, (2, v))) ws = other.map(lambda (k, v): (k, (2, v)))
return vs.union(ws).groupByKey(numSplits).flatMapValues(dispatch) return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch)
def python_join(rdd, other, numSplits): def python_join(rdd, other, numPartitions):
def dispatch(seq): def dispatch(seq):
vbuf, wbuf = [], [] vbuf, wbuf = [], []
for (n, v) in seq: for (n, v) in seq:
@ -47,10 +47,10 @@ def python_join(rdd, other, numSplits):
elif n == 2: elif n == 2:
wbuf.append(v) wbuf.append(v)
return [(v, w) for v in vbuf for w in wbuf] return [(v, w) for v in vbuf for w in wbuf]
return _do_python_join(rdd, other, numSplits, dispatch) return _do_python_join(rdd, other, numPartitions, dispatch)
def python_right_outer_join(rdd, other, numSplits): def python_right_outer_join(rdd, other, numPartitions):
def dispatch(seq): def dispatch(seq):
vbuf, wbuf = [], [] vbuf, wbuf = [], []
for (n, v) in seq: for (n, v) in seq:
@ -61,10 +61,10 @@ def python_right_outer_join(rdd, other, numSplits):
if not vbuf: if not vbuf:
vbuf.append(None) vbuf.append(None)
return [(v, w) for v in vbuf for w in wbuf] return [(v, w) for v in vbuf for w in wbuf]
return _do_python_join(rdd, other, numSplits, dispatch) return _do_python_join(rdd, other, numPartitions, dispatch)
def python_left_outer_join(rdd, other, numSplits): def python_left_outer_join(rdd, other, numPartitions):
def dispatch(seq): def dispatch(seq):
vbuf, wbuf = [], [] vbuf, wbuf = [], []
for (n, v) in seq: for (n, v) in seq:
@ -75,10 +75,10 @@ def python_left_outer_join(rdd, other, numSplits):
if not wbuf: if not wbuf:
wbuf.append(None) wbuf.append(None)
return [(v, w) for v in vbuf for w in wbuf] return [(v, w) for v in vbuf for w in wbuf]
return _do_python_join(rdd, other, numSplits, dispatch) return _do_python_join(rdd, other, numPartitions, dispatch)
def python_cogroup(rdd, other, numSplits): def python_cogroup(rdd, other, numPartitions):
vs = rdd.map(lambda (k, v): (k, (1, v))) vs = rdd.map(lambda (k, v): (k, (1, v)))
ws = other.map(lambda (k, v): (k, (2, v))) ws = other.map(lambda (k, v): (k, (2, v)))
def dispatch(seq): def dispatch(seq):
@ -89,4 +89,4 @@ def python_cogroup(rdd, other, numSplits):
elif n == 2: elif n == 2:
wbuf.append(v) wbuf.append(v)
return (vbuf, wbuf) return (vbuf, wbuf)
return vs.union(ws).groupByKey(numSplits).mapValues(dispatch) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch)

View file

@ -215,7 +215,7 @@ class RDD(object):
yield pair yield pair
return java_cartesian.flatMap(unpack_batches) return java_cartesian.flatMap(unpack_batches)
def groupBy(self, f, numSplits=None): def groupBy(self, f, numPartitions=None):
""" """
Return an RDD of grouped items. Return an RDD of grouped items.
@ -224,7 +224,7 @@ class RDD(object):
>>> sorted([(x, sorted(y)) for (x, y) in result]) >>> sorted([(x, sorted(y)) for (x, y) in result])
[(0, [2, 8]), (1, [1, 1, 3, 5])] [(0, [2, 8]), (1, [1, 1, 3, 5])]
""" """
return self.map(lambda x: (f(x), x)).groupByKey(numSplits) return self.map(lambda x: (f(x), x)).groupByKey(numPartitions)
def pipe(self, command, env={}): def pipe(self, command, env={}):
""" """
@ -274,7 +274,7 @@ class RDD(object):
def reduce(self, f): def reduce(self, f):
""" """
Reduces the elements of this RDD using the specified commutative and Reduces the elements of this RDD using the specified commutative and
associative binary operator. associative binary operator.
>>> from operator import add >>> from operator import add
@ -422,22 +422,22 @@ class RDD(object):
""" """
return dict(self.collect()) return dict(self.collect())
def reduceByKey(self, func, numSplits=None): def reduceByKey(self, func, numPartitions=None):
""" """
Merge the values for each key using an associative reduce function. Merge the values for each key using an associative reduce function.
This will also perform the merging locally on each mapper before This will also perform the merging locally on each mapper before
sending results to a reducer, similarly to a "combiner" in MapReduce. sending results to a reducer, similarly to a "combiner" in MapReduce.
Output will be hash-partitioned with C{numSplits} splits, or the Output will be hash-partitioned with C{numPartitions} partitions, or
default parallelism level if C{numSplits} is not specified. the default parallelism level if C{numPartitions} is not specified.
>>> from operator import add >>> from operator import add
>>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
>>> sorted(rdd.reduceByKey(add).collect()) >>> sorted(rdd.reduceByKey(add).collect())
[('a', 2), ('b', 1)] [('a', 2), ('b', 1)]
""" """
return self.combineByKey(lambda x: x, func, func, numSplits) return self.combineByKey(lambda x: x, func, func, numPartitions)
def reduceByKeyLocally(self, func): def reduceByKeyLocally(self, func):
""" """
@ -474,7 +474,7 @@ class RDD(object):
""" """
return self.map(lambda x: x[0]).countByValue() return self.map(lambda x: x[0]).countByValue()
def join(self, other, numSplits=None): def join(self, other, numPartitions=None):
""" """
Return an RDD containing all pairs of elements with matching keys in Return an RDD containing all pairs of elements with matching keys in
C{self} and C{other}. C{self} and C{other}.
@ -489,9 +489,9 @@ class RDD(object):
>>> sorted(x.join(y).collect()) >>> sorted(x.join(y).collect())
[('a', (1, 2)), ('a', (1, 3))] [('a', (1, 2)), ('a', (1, 3))]
""" """
return python_join(self, other, numSplits) return python_join(self, other, numPartitions)
def leftOuterJoin(self, other, numSplits=None): def leftOuterJoin(self, other, numPartitions=None):
""" """
Perform a left outer join of C{self} and C{other}. Perform a left outer join of C{self} and C{other}.
@ -506,9 +506,9 @@ class RDD(object):
>>> sorted(x.leftOuterJoin(y).collect()) >>> sorted(x.leftOuterJoin(y).collect())
[('a', (1, 2)), ('b', (4, None))] [('a', (1, 2)), ('b', (4, None))]
""" """
return python_left_outer_join(self, other, numSplits) return python_left_outer_join(self, other, numPartitions)
def rightOuterJoin(self, other, numSplits=None): def rightOuterJoin(self, other, numPartitions=None):
""" """
Perform a right outer join of C{self} and C{other}. Perform a right outer join of C{self} and C{other}.
@ -523,10 +523,10 @@ class RDD(object):
>>> sorted(y.rightOuterJoin(x).collect()) >>> sorted(y.rightOuterJoin(x).collect())
[('a', (2, 1)), ('b', (None, 4))] [('a', (2, 1)), ('b', (None, 4))]
""" """
return python_right_outer_join(self, other, numSplits) return python_right_outer_join(self, other, numPartitions)
# TODO: add option to control map-side combining # TODO: add option to control map-side combining
def partitionBy(self, numSplits, partitionFunc=hash): def partitionBy(self, numPartitions, partitionFunc=hash):
""" """
Return a copy of the RDD partitioned using the specified partitioner. Return a copy of the RDD partitioned using the specified partitioner.
@ -535,22 +535,22 @@ class RDD(object):
>>> set(sets[0]).intersection(set(sets[1])) >>> set(sets[0]).intersection(set(sets[1]))
set([]) set([])
""" """
if numSplits is None: if numPartitions is None:
numSplits = self.ctx.defaultParallelism numPartitions = self.ctx.defaultParallelism
# Transferring O(n) objects to Java is too expensive. Instead, we'll # Transferring O(n) objects to Java is too expensive. Instead, we'll
# form the hash buckets in Python, transferring O(numSplits) objects # form the hash buckets in Python, transferring O(numPartitions) objects
# to Java. Each object is a (splitNumber, [objects]) pair. # to Java. Each object is a (splitNumber, [objects]) pair.
def add_shuffle_key(split, iterator): def add_shuffle_key(split, iterator):
buckets = defaultdict(list) buckets = defaultdict(list)
for (k, v) in iterator: for (k, v) in iterator:
buckets[partitionFunc(k) % numSplits].append((k, v)) buckets[partitionFunc(k) % numPartitions].append((k, v))
for (split, items) in buckets.iteritems(): for (split, items) in buckets.iteritems():
yield str(split) yield str(split)
yield dump_pickle(Batch(items)) yield dump_pickle(Batch(items))
keyed = PipelinedRDD(self, add_shuffle_key) keyed = PipelinedRDD(self, add_shuffle_key)
keyed._bypass_serializer = True keyed._bypass_serializer = True
pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
partitioner = self.ctx._jvm.PythonPartitioner(numSplits, partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
id(partitionFunc)) id(partitionFunc))
jrdd = pairRDD.partitionBy(partitioner).values() jrdd = pairRDD.partitionBy(partitioner).values()
rdd = RDD(jrdd, self.ctx) rdd = RDD(jrdd, self.ctx)
@ -561,7 +561,7 @@ class RDD(object):
# TODO: add control over map-side aggregation # TODO: add control over map-side aggregation
def combineByKey(self, createCombiner, mergeValue, mergeCombiners, def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
numSplits=None): numPartitions=None):
""" """
Generic function to combine the elements for each key using a custom Generic function to combine the elements for each key using a custom
set of aggregation functions. set of aggregation functions.
@ -586,8 +586,8 @@ class RDD(object):
>>> sorted(x.combineByKey(str, add, add).collect()) >>> sorted(x.combineByKey(str, add, add).collect())
[('a', '11'), ('b', '1')] [('a', '11'), ('b', '1')]
""" """
if numSplits is None: if numPartitions is None:
numSplits = self.ctx.defaultParallelism numPartitions = self.ctx.defaultParallelism
def combineLocally(iterator): def combineLocally(iterator):
combiners = {} combiners = {}
for (k, v) in iterator: for (k, v) in iterator:
@ -597,7 +597,7 @@ class RDD(object):
combiners[k] = mergeValue(combiners[k], v) combiners[k] = mergeValue(combiners[k], v)
return combiners.iteritems() return combiners.iteritems()
locally_combined = self.mapPartitions(combineLocally) locally_combined = self.mapPartitions(combineLocally)
shuffled = locally_combined.partitionBy(numSplits) shuffled = locally_combined.partitionBy(numPartitions)
def _mergeCombiners(iterator): def _mergeCombiners(iterator):
combiners = {} combiners = {}
for (k, v) in iterator: for (k, v) in iterator:
@ -609,10 +609,10 @@ class RDD(object):
return shuffled.mapPartitions(_mergeCombiners) return shuffled.mapPartitions(_mergeCombiners)
# TODO: support variant with custom partitioner # TODO: support variant with custom partitioner
def groupByKey(self, numSplits=None): def groupByKey(self, numPartitions=None):
""" """
Group the values for each key in the RDD into a single sequence. Group the values for each key in the RDD into a single sequence.
Hash-partitions the resulting RDD with into numSplits partitions. Hash-partitions the resulting RDD with into numPartitions partitions.
>>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
>>> sorted(x.groupByKey().collect()) >>> sorted(x.groupByKey().collect())
@ -630,7 +630,7 @@ class RDD(object):
return a + b return a + b
return self.combineByKey(createCombiner, mergeValue, mergeCombiners, return self.combineByKey(createCombiner, mergeValue, mergeCombiners,
numSplits) numPartitions)
# TODO: add tests # TODO: add tests
def flatMapValues(self, f): def flatMapValues(self, f):
@ -659,7 +659,7 @@ class RDD(object):
return self.cogroup(other) return self.cogroup(other)
# TODO: add variant with custom parittioner # TODO: add variant with custom parittioner
def cogroup(self, other, numSplits=None): def cogroup(self, other, numPartitions=None):
""" """
For each key k in C{self} or C{other}, return a resulting RDD that For each key k in C{self} or C{other}, return a resulting RDD that
contains a tuple with the list of values for that key in C{self} as well contains a tuple with the list of values for that key in C{self} as well
@ -670,7 +670,7 @@ class RDD(object):
>>> sorted(x.cogroup(y).collect()) >>> sorted(x.cogroup(y).collect())
[('a', ([1], [2])), ('b', ([4], []))] [('a', ([1], [2])), ('b', ([4], []))]
""" """
return python_cogroup(self, other, numSplits) return python_cogroup(self, other, numPartitions)
# TODO: `lookup` is disabled because we can't make direct comparisons based # TODO: `lookup` is disabled because we can't make direct comparisons based
# on the key; we need to compare the hash of the key to the hash of the # on the key; we need to compare the hash of the key to the hash of the

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>

View file

@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
____ __ ____ __
/ __/__ ___ _____/ /__ / __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/ _\ \/ _ \/ _ `/ __/ '_/
/___/ .__/\_,_/_/ /_/\_\ version 0.7.0 /___/ .__/\_,_/_/ /_/\_\ version 0.7.1
/_/ /_/
""") """)
import Properties._ import Properties._

33
run
View file

@ -25,6 +25,26 @@ if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker"
SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default
fi fi
# Add java opts for master, worker, executor. The opts maybe null
case "$1" in
'spark.deploy.master.Master')
SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS"
;;
'spark.deploy.worker.Worker')
SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS"
;;
'spark.executor.StandaloneExecutorBackend')
SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
;;
'spark.executor.MesosExecutorBackend')
SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
;;
'spark.repl.Main')
SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS"
;;
esac
if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
if [ `command -v scala` ]; then if [ `command -v scala` ]; then
RUNNER="scala" RUNNER="scala"
@ -91,11 +111,13 @@ CLASSPATH+=":$FWDIR/conf"
CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes"
if [ -n "$SPARK_TESTING" ] ; then if [ -n "$SPARK_TESTING" ] ; then
CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
fi fi
CLASSPATH+=":$CORE_DIR/src/main/resources" CLASSPATH+=":$CORE_DIR/src/main/resources"
CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
if [ -e "$FWDIR/lib_managed" ]; then if [ -e "$FWDIR/lib_managed" ]; then
CLASSPATH+=":$FWDIR/lib_managed/jars/*" CLASSPATH+=":$FWDIR/lib_managed/jars/*"
CLASSPATH+=":$FWDIR/lib_managed/bundles/*" CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
@ -112,6 +134,17 @@ for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
done done
export CLASSPATH # Needed for spark-shell export CLASSPATH # Needed for spark-shell
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
# to avoid the -sources and -doc packages that are built by publish-local.
if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
# Use the JAR from the SBT build
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
fi
if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then
# Use the JAR from the Maven build
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar`
fi
# Figure out whether to run our class with java or with the scala launcher. # Figure out whether to run our class with java or with the scala launcher.
# In most cases, we'd prefer to execute our process with java because scala # In most cases, we'd prefer to execute our process with java because scala
# creates a shell script as the parent of its Java process, which makes it # creates a shell script as the parent of its Java process, which makes it

View file

@ -47,11 +47,14 @@ set CORE_DIR=%FWDIR%core
set REPL_DIR=%FWDIR%repl set REPL_DIR=%FWDIR%repl
set EXAMPLES_DIR=%FWDIR%examples set EXAMPLES_DIR=%FWDIR%examples
set BAGEL_DIR=%FWDIR%bagel set BAGEL_DIR=%FWDIR%bagel
set STREAMING_DIR=%FWDIR%streaming
set PYSPARK_DIR=%FWDIR%python set PYSPARK_DIR=%FWDIR%python
rem Build up classpath rem Build up classpath
set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources
set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes
set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\*
set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\*
set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\*
@ -59,6 +62,16 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\*
set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\*
set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
rem Figure out the JAR file that our examples were packaged into.
rem First search in the build path from SBT:
for %%d in ("examples/target/scala-%SCALA_VERSION%/spark-examples*.jar") do (
set SPARK_EXAMPLES_JAR=examples/target/scala-%SCALA_VERSION%/%%d
)
rem Then search in the build path from Maven:
for %%d in ("examples/target/spark-examples*hadoop*.jar") do (
set SPARK_EXAMPLES_JAR=examples/target/%%d
)
rem Figure out whether to run our class with java or with the scala launcher. rem Figure out whether to run our class with java or with the scala launcher.
rem In most cases, we'd prefer to execute our process with java because scala rem In most cases, we'd prefer to execute our process with java because scala
rem creates a shell script as the parent of its Java process, which makes it rem creates a shell script as the parent of its Java process, which makes it

View file

@ -4,7 +4,7 @@
<parent> <parent>
<groupId>org.spark-project</groupId> <groupId>org.spark-project</groupId>
<artifactId>parent</artifactId> <artifactId>parent</artifactId>
<version>0.7.0-SNAPSHOT</version> <version>0.7.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath> <relativePath>../pom.xml</relativePath>
</parent> </parent>
@ -47,7 +47,16 @@
<artifactId>zkclient</artifactId> <artifactId>zkclient</artifactId>
<version>0.1</version> <version>0.1</version>
</dependency> </dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-stream</artifactId>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
<artifactId>akka-zeromq</artifactId>
<version>2.0.3</version>
</dependency>
<dependency> <dependency>
<groupId>org.scalatest</groupId> <groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.version}</artifactId> <artifactId>scalatest_${scala.version}</artifactId>

View file

@ -6,6 +6,8 @@ import org.apache.hadoop.fs.{FileUtil, Path}
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import java.io._ import java.io._
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
import java.util.concurrent.Executors
private[streaming] private[streaming]
@ -15,9 +17,11 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
val framework = ssc.sc.appName val framework = ssc.sc.appName
val sparkHome = ssc.sc.sparkHome val sparkHome = ssc.sc.sparkHome
val jars = ssc.sc.jars val jars = ssc.sc.jars
val environment = ssc.sc.environment
val graph = ssc.graph val graph = ssc.graph
val checkpointDir = ssc.checkpointDir val checkpointDir = ssc.checkpointDir
val checkpointDuration: Duration = ssc.checkpointDuration val checkpointDuration = ssc.checkpointDuration
val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
def validate() { def validate() {
assert(master != null, "Checkpoint.master is null") assert(master != null, "Checkpoint.master is null")
@ -37,32 +41,50 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
val conf = new Configuration() val conf = new Configuration()
var fs = file.getFileSystem(conf) var fs = file.getFileSystem(conf)
val maxAttempts = 3 val maxAttempts = 3
val executor = Executors.newFixedThreadPool(1)
class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
def run() {
var attempts = 0
val startTime = System.currentTimeMillis()
while (attempts < maxAttempts) {
attempts += 1
try {
logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
if (fs.exists(file)) {
val bkFile = new Path(file.getParent, file.getName + ".bk")
FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
logDebug("Moved existing checkpoint file to " + bkFile)
}
val fos = fs.create(file)
fos.write(bytes)
fos.close()
fos.close()
val finishTime = System.currentTimeMillis();
logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
"', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
return
} catch {
case ioe: IOException =>
logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
}
}
logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'")
}
}
def write(checkpoint: Checkpoint) { def write(checkpoint: Checkpoint) {
// TODO: maybe do this in a different thread from the main stream execution thread val bos = new ByteArrayOutputStream()
var attempts = 0 val zos = new LZFOutputStream(bos)
while (attempts < maxAttempts) { val oos = new ObjectOutputStream(zos)
attempts += 1 oos.writeObject(checkpoint)
try { oos.close()
logDebug("Saving checkpoint for time " + checkpoint.checkpointTime + " to file '" + file + "'") bos.close()
if (fs.exists(file)) { executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray))
val bkFile = new Path(file.getParent, file.getName + ".bk") }
FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
logDebug("Moved existing checkpoint file to " + bkFile) def stop() {
} executor.shutdown()
val fos = fs.create(file)
val oos = new ObjectOutputStream(fos)
oos.writeObject(checkpoint)
oos.close()
logInfo("Checkpoint for time " + checkpoint.checkpointTime + " saved to file '" + file + "'")
fos.close()
return
} catch {
case ioe: IOException =>
logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
}
}
logError("Could not write checkpoint for time " + checkpoint.checkpointTime + " to file '" + file + "'")
} }
} }
@ -84,7 +106,8 @@ object CheckpointReader extends Logging {
// of ObjectInputStream is used to explicitly use the current thread's default class // of ObjectInputStream is used to explicitly use the current thread's default class
// loader to find and load classes. This is a well know Java issue and has popped up // loader to find and load classes. This is a well know Java issue and has popped up
// in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
val ois = new ObjectInputStreamWithLoader(fis, Thread.currentThread().getContextClassLoader) val zis = new LZFInputStream(fis)
val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
val cp = ois.readObject.asInstanceOf[Checkpoint] val cp = ois.readObject.asInstanceOf[Checkpoint]
ois.close() ois.close()
fs.close() fs.close()

View file

@ -12,7 +12,7 @@ import scala.collection.mutable.HashMap
import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
/** /**
@ -75,7 +75,7 @@ abstract class DStream[T: ClassManifest] (
// Checkpoint details // Checkpoint details
protected[streaming] val mustCheckpoint = false protected[streaming] val mustCheckpoint = false
protected[streaming] var checkpointDuration: Duration = null protected[streaming] var checkpointDuration: Duration = null
protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]()) protected[streaming] val checkpointData = new DStreamCheckpointData(this)
// Reference to whole DStream graph // Reference to whole DStream graph
protected[streaming] var graph: DStreamGraph = null protected[streaming] var graph: DStreamGraph = null
@ -85,10 +85,10 @@ abstract class DStream[T: ClassManifest] (
// Duration for which the DStream requires its parent DStream to remember each RDD created // Duration for which the DStream requires its parent DStream to remember each RDD created
protected[streaming] def parentRememberDuration = rememberDuration protected[streaming] def parentRememberDuration = rememberDuration
/** Returns the StreamingContext associated with this DStream */ /** Return the StreamingContext associated with this DStream */
def context() = ssc def context = ssc
/** Persists the RDDs of this DStream with the given storage level */ /** Persist the RDDs of this DStream with the given storage level */
def persist(level: StorageLevel): DStream[T] = { def persist(level: StorageLevel): DStream[T] = {
if (this.isInitialized) { if (this.isInitialized) {
throw new UnsupportedOperationException( throw new UnsupportedOperationException(
@ -132,7 +132,7 @@ abstract class DStream[T: ClassManifest] (
// Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger
if (mustCheckpoint && checkpointDuration == null) { if (mustCheckpoint && checkpointDuration == null) {
checkpointDuration = slideDuration.max(Seconds(10)) checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt
logInfo("Checkpoint interval automatically set to " + checkpointDuration) logInfo("Checkpoint interval automatically set to " + checkpointDuration)
} }
@ -159,7 +159,7 @@ abstract class DStream[T: ClassManifest] (
) )
assert( assert(
checkpointDuration == null || ssc.sc.checkpointDir.isDefined, checkpointDuration == null || context.sparkContext.checkpointDir.isDefined,
"The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" + "The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" +
" or SparkContext.checkpoint() to set the checkpoint directory." " or SparkContext.checkpoint() to set the checkpoint directory."
) )
@ -238,13 +238,15 @@ abstract class DStream[T: ClassManifest] (
dependencies.foreach(_.remember(parentRememberDuration)) dependencies.foreach(_.remember(parentRememberDuration))
} }
/** This method checks whether the 'time' is valid wrt slideDuration for generating RDD */ /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */
protected def isTimeValid(time: Time): Boolean = { protected def isTimeValid(time: Time): Boolean = {
if (!isInitialized) { if (!isInitialized) {
throw new Exception (this + " has not been initialized") throw new Exception (this + " has not been initialized")
} else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) {
logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime))
false false
} else { } else {
logInfo("Time " + time + " is valid")
true true
} }
} }
@ -292,14 +294,14 @@ abstract class DStream[T: ClassManifest] (
* Generate a SparkStreaming job for the given time. This is an internal method that * Generate a SparkStreaming job for the given time. This is an internal method that
* should not be called directly. This default implementation creates a job * should not be called directly. This default implementation creates a job
* that materializes the corresponding RDD. Subclasses of DStream may override this * that materializes the corresponding RDD. Subclasses of DStream may override this
* (eg. ForEachDStream). * to generate their own jobs.
*/ */
protected[streaming] def generateJob(time: Time): Option[Job] = { protected[streaming] def generateJob(time: Time): Option[Job] = {
getOrCompute(time) match { getOrCompute(time) match {
case Some(rdd) => { case Some(rdd) => {
val jobFunc = () => { val jobFunc = () => {
val emptyFunc = { (iterator: Iterator[T]) => {} } val emptyFunc = { (iterator: Iterator[T]) => {} }
ssc.sc.runJob(rdd, emptyFunc) context.sparkContext.runJob(rdd, emptyFunc)
} }
Some(new Job(time, jobFunc)) Some(new Job(time, jobFunc))
} }
@ -308,20 +310,18 @@ abstract class DStream[T: ClassManifest] (
} }
/** /**
* Dereference RDDs that are older than rememberDuration. * Clear metadata that are older than `rememberDuration` of this DStream.
* This is an internal method that should not be called directly. This default
* implementation clears the old generated RDDs. Subclasses of DStream may override
* this to clear their own metadata along with the generated RDDs.
*/ */
protected[streaming] def forgetOldRDDs(time: Time) { protected[streaming] def clearOldMetadata(time: Time) {
val keys = generatedRDDs.keys
var numForgotten = 0 var numForgotten = 0
keys.foreach(t => { val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration))
if (t <= (time - rememberDuration)) { generatedRDDs --= oldRDDs.keys
generatedRDDs.remove(t) logInfo("Cleared " + oldRDDs.size + " RDDs that were older than " +
numForgotten += 1 (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", "))
logInfo("Forgot RDD of time " + t + " from " + this) dependencies.foreach(_.clearOldMetadata(time))
}
})
logInfo("Forgot " + numForgotten + " RDDs from " + this)
dependencies.foreach(_.forgetOldRDDs(time))
} }
/* Adds metadata to the Stream while it is running. /* Adds metadata to the Stream while it is running.
@ -342,40 +342,10 @@ abstract class DStream[T: ClassManifest] (
*/ */
protected[streaming] def updateCheckpointData(currentTime: Time) { protected[streaming] def updateCheckpointData(currentTime: Time) {
logInfo("Updating checkpoint data for time " + currentTime) logInfo("Updating checkpoint data for time " + currentTime)
checkpointData.update()
// Get the checkpointed RDDs from the generated RDDs
val newRdds = generatedRDDs.filter(_._2.getCheckpointFile.isDefined)
.map(x => (x._1, x._2.getCheckpointFile.get))
// Make a copy of the existing checkpoint data (checkpointed RDDs)
val oldRdds = checkpointData.rdds.clone()
// If the new checkpoint data has checkpoints then replace existing with the new one
if (newRdds.size > 0) {
checkpointData.rdds.clear()
checkpointData.rdds ++= newRdds
}
// Make parent DStreams update their checkpoint data
dependencies.foreach(_.updateCheckpointData(currentTime)) dependencies.foreach(_.updateCheckpointData(currentTime))
checkpointData.cleanup()
// TODO: remove this, this is just for debugging logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData)
newRdds.foreach {
case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") }
}
if (newRdds.size > 0) {
(oldRdds -- newRdds.keySet).foreach {
case (time, data) => {
val path = new Path(data.toString)
val fs = path.getFileSystem(new Configuration())
fs.delete(path, true)
logInfo("Deleted checkpoint file '" + path + "' for time " + time)
}
}
}
logInfo("Updated checkpoint data for time " + currentTime + ", " + checkpointData.rdds.size + " checkpoints, "
+ "[" + checkpointData.rdds.mkString(",") + "]")
} }
/** /**
@ -386,14 +356,8 @@ abstract class DStream[T: ClassManifest] (
*/ */
protected[streaming] def restoreCheckpointData() { protected[streaming] def restoreCheckpointData() {
// Create RDDs from the checkpoint data // Create RDDs from the checkpoint data
logInfo("Restoring checkpoint data from " + checkpointData.rdds.size + " checkpointed RDDs") logInfo("Restoring checkpoint data")
checkpointData.rdds.foreach { checkpointData.restore()
case(time, data) => {
logInfo("Restoring checkpointed RDD for time " + time + " from file '" + data.toString + "'")
val rdd = ssc.sc.checkpointFile[T](data.toString)
generatedRDDs += ((time, rdd))
}
}
dependencies.foreach(_.restoreCheckpointData()) dependencies.foreach(_.restoreCheckpointData())
logInfo("Restored checkpoint data") logInfo("Restored checkpoint data")
} }
@ -433,7 +397,7 @@ abstract class DStream[T: ClassManifest] (
/** Return a new DStream by applying a function to all elements of this DStream. */ /** 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: ClassManifest](mapFunc: T => U): DStream[U] = {
new MappedDStream(this, ssc.sc.clean(mapFunc)) new MappedDStream(this, context.sparkContext.clean(mapFunc))
} }
/** /**
@ -441,7 +405,7 @@ abstract class DStream[T: ClassManifest] (
* and then flattening the results * and then flattening the results
*/ */
def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = { def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
new FlatMappedDStream(this, ssc.sc.clean(flatMapFunc)) new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc))
} }
/** Return a new DStream containing only the elements that satisfy a predicate. */ /** Return a new DStream containing only the elements that satisfy a predicate. */
@ -463,7 +427,7 @@ abstract class DStream[T: ClassManifest] (
mapPartFunc: Iterator[T] => Iterator[U], mapPartFunc: Iterator[T] => Iterator[U],
preservePartitioning: Boolean = false preservePartitioning: Boolean = false
): DStream[U] = { ): DStream[U] = {
new MapPartitionedDStream(this, ssc.sc.clean(mapPartFunc), preservePartitioning) new MapPartitionedDStream(this, context.sparkContext.clean(mapPartFunc), preservePartitioning)
} }
/** /**
@ -479,6 +443,15 @@ abstract class DStream[T: ClassManifest] (
*/ */
def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _) def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _)
/**
* Return a new DStream in which each RDD contains the counts of each distinct value in
* each RDD of this DStream. Hash partitioning is used to generate
* the RDDs with `numPartitions` partitions (Spark's default number of partitions if
* `numPartitions` not specified).
*/
def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] =
this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
/** /**
* Apply a function to each RDD in this DStream. This is an output operator, so * Apply a function to each RDD in this DStream. This is an output operator, so
* this DStream will be registered as an output stream and therefore materialized. * this DStream will be registered as an output stream and therefore materialized.
@ -492,7 +465,7 @@ abstract class DStream[T: ClassManifest] (
* this DStream will be registered as an output stream and therefore materialized. * this DStream will be registered as an output stream and therefore materialized.
*/ */
def foreach(foreachFunc: (RDD[T], Time) => Unit) { def foreach(foreachFunc: (RDD[T], Time) => Unit) {
val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc)) val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
ssc.registerOutputStream(newStream) ssc.registerOutputStream(newStream)
newStream newStream
} }
@ -510,7 +483,7 @@ abstract class DStream[T: ClassManifest] (
* on each RDD of this DStream. * on each RDD of this DStream.
*/ */
def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
new TransformedDStream(this, ssc.sc.clean(transformFunc)) new TransformedDStream(this, context.sparkContext.clean(transformFunc))
} }
/** /**
@ -527,19 +500,21 @@ abstract class DStream[T: ClassManifest] (
if (first11.size > 10) println("...") if (first11.size > 10) println("...")
println() println()
} }
val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc)) val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
ssc.registerOutputStream(newStream) ssc.registerOutputStream(newStream)
} }
/** /**
* Return a new DStream which is computed based on windowed batches of this DStream. * Return a new DStream in which each RDD contains all the elements in seen in a
* The new DStream generates RDDs with the same interval as this DStream. * sliding window of time over this DStream. The new DStream generates RDDs with
* the same interval as this DStream.
* @param windowDuration width of the window; must be a multiple of this DStream's interval. * @param windowDuration width of the window; must be a multiple of this DStream's interval.
*/ */
def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration) def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration)
/** /**
* Return a new DStream which is computed based on windowed batches of this DStream. * Return a new DStream in which each RDD contains all the elements in seen in a
* sliding window of time over this DStream.
* @param windowDuration width of the window; must be a multiple of this DStream's * @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval * batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which * @param slideDuration sliding interval of the window (i.e., the interval after which
@ -550,28 +525,40 @@ abstract class DStream[T: ClassManifest] (
new WindowedDStream(this, windowDuration, slideDuration) new WindowedDStream(this, windowDuration, slideDuration)
} }
/**
* Return a new DStream which computed based on tumbling window on this DStream.
* This is equivalent to window(batchTime, batchTime).
* @param batchDuration tumbling window duration; must be a multiple of this DStream's
* batching interval
*/
def tumble(batchDuration: Duration): DStream[T] = window(batchDuration, batchDuration)
/** /**
* Return a new DStream in which each RDD has a single element generated by reducing all * Return a new DStream in which each RDD has a single element generated by reducing all
* elements in a window over this DStream. windowDuration and slideDuration are as defined * elements in a sliding window over this DStream.
* in the window() operation. This is equivalent to * @param reduceFunc associative reduce function
* window(windowDuration, slideDuration).reduce(reduceFunc) * @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
*/ */
def reduceByWindow( def reduceByWindow(
reduceFunc: (T, T) => T, reduceFunc: (T, T) => T,
windowDuration: Duration, windowDuration: Duration,
slideDuration: Duration slideDuration: Duration
): DStream[T] = { ): DStream[T] = {
this.window(windowDuration, slideDuration).reduce(reduceFunc) this.reduce(reduceFunc).window(windowDuration, slideDuration).reduce(reduceFunc)
} }
/**
* Return a new DStream in which each RDD has a single element generated by reducing all
* elements in a sliding window over this DStream. However, the reduction is done incrementally
* using the old window's reduced value :
* 1. reduce the new values that entered the window (e.g., adding new counts)
* 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
* This is more efficient than reduceByWindow without "inverse reduce" function.
* However, it is applicable to only "invertible reduce functions".
* @param reduceFunc associative reduce function
* @param invReduceFunc inverse reduce function
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
*/
def reduceByWindow( def reduceByWindow(
reduceFunc: (T, T) => T, reduceFunc: (T, T) => T,
invReduceFunc: (T, T) => T, invReduceFunc: (T, T) => T,
@ -585,13 +572,46 @@ abstract class DStream[T: ClassManifest] (
/** /**
* Return a new DStream in which each RDD has a single element generated by counting the number * Return a new DStream in which each RDD has a single element generated by counting the number
* of elements in a window over this DStream. windowDuration and slideDuration are as defined in the * of elements in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with
* window() operation. This is equivalent to window(windowDuration, slideDuration).count() * Spark's default number of partitions.
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
*/ */
def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = { def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = {
this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration) this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration)
} }
/**
* Return a new DStream in which each RDD contains the count of distinct elements in
* RDDs in a sliding window over this DStream. Hash partitioning is used to generate
* the RDDs with `numPartitions` partitions (Spark's default number of partitions if
* `numPartitions` not specified).
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
* @param numPartitions number of partitions of each RDD in the new DStream.
*/
def countByValueAndWindow(
windowDuration: Duration,
slideDuration: Duration,
numPartitions: Int = ssc.sc.defaultParallelism
): DStream[(T, Long)] = {
this.map(x => (x, 1L)).reduceByKeyAndWindow(
(x: Long, y: Long) => x + y,
(x: Long, y: Long) => x - y,
windowDuration,
slideDuration,
numPartitions,
(x: (T, Long)) => x._2 != 0L
)
}
/** /**
* Return a new DStream by unifying data of another DStream with this DStream. * Return a new DStream by unifying data of another DStream with this DStream.
* @param that Another DStream having the same slideDuration as this DStream. * @param that Another DStream having the same slideDuration as this DStream.
@ -609,16 +629,21 @@ abstract class DStream[T: ClassManifest] (
* Return all the RDDs between 'fromTime' to 'toTime' (both included) * Return all the RDDs between 'fromTime' to 'toTime' (both included)
*/ */
def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = {
val rdds = new ArrayBuffer[RDD[T]]() if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) {
var time = toTime.floor(slideDuration) logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")")
while (time >= zeroTime && time >= fromTime) {
getOrCompute(time) match {
case Some(rdd) => rdds += rdd
case None => //throw new Exception("Could not get RDD for time " + time)
}
time -= slideDuration
} }
rdds.toSeq if (!(toTime - zeroTime).isMultipleOf(slideDuration)) {
logWarning("toTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")")
}
val alignedToTime = toTime.floor(slideDuration)
val alignedFromTime = fromTime.floor(slideDuration)
logInfo("Slicing from " + fromTime + " to " + toTime +
" (aligned to " + alignedFromTime + " and " + alignedToTime + ")")
alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => {
if (time >= zeroTime) getOrCompute(time) else None
})
} }
/** /**
@ -651,7 +676,3 @@ abstract class DStream[T: ClassManifest] (
ssc.registerOutputStream(this) ssc.registerOutputStream(this)
} }
} }
private[streaming]
case class DStreamCheckpointData(rdds: HashMap[Time, Any])

View file

@ -0,0 +1,93 @@
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
private[streaming]
class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
extends Serializable with Logging {
protected val data = new HashMap[Time, AnyRef]()
@transient private var fileSystem : FileSystem = null
@transient private var lastCheckpointFiles: HashMap[Time, String] = null
protected[streaming] def checkpointFiles = data.asInstanceOf[HashMap[Time, String]]
/**
* Updates the checkpoint data of the DStream. This gets called every time
* the graph checkpoint is initiated. Default implementation records the
* checkpoint files to which the generate RDDs of the DStream has been saved.
*/
def update() {
// Get the checkpointed RDDs from the generated RDDs
val newCheckpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined)
.map(x => (x._1, x._2.getCheckpointFile.get))
// Make a copy of the existing checkpoint data (checkpointed RDDs)
lastCheckpointFiles = checkpointFiles.clone()
// If the new checkpoint data has checkpoints then replace existing with the new one
if (newCheckpointFiles.size > 0) {
checkpointFiles.clear()
checkpointFiles ++= newCheckpointFiles
}
// TODO: remove this, this is just for debugging
newCheckpointFiles.foreach {
case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") }
}
}
/**
* Cleanup old checkpoint data. This gets called every time the graph
* checkpoint is initiated, but after `update` is called. Default
* implementation, cleans up old checkpoint files.
*/
def cleanup() {
// If there is at least on checkpoint file in the current checkpoint files,
// then delete the old checkpoint files.
if (checkpointFiles.size > 0 && lastCheckpointFiles != null) {
(lastCheckpointFiles -- checkpointFiles.keySet).foreach {
case (time, file) => {
try {
val path = new Path(file)
if (fileSystem == null) {
fileSystem = path.getFileSystem(new Configuration())
}
fileSystem.delete(path, true)
logInfo("Deleted checkpoint file '" + file + "' for time " + time)
} catch {
case e: Exception =>
logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e)
}
}
}
}
}
/**
* Restore the checkpoint data. This gets called once when the DStream graph
* (along with its DStreams) are being restored from a graph checkpoint file.
* Default implementation restores the RDDs from their checkpoint files.
*/
def restore() {
// Create RDDs from the checkpoint data
checkpointFiles.foreach {
case(time, file) => {
logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'")
dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file)))
}
}
}
override def toString() = {
"[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]"
}
}

View file

@ -11,17 +11,20 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
private val inputStreams = new ArrayBuffer[InputDStream[_]]() private val inputStreams = new ArrayBuffer[InputDStream[_]]()
private val outputStreams = new ArrayBuffer[DStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]()
private[streaming] var zeroTime: Time = null var rememberDuration: Duration = null
private[streaming] var batchDuration: Duration = null var checkpointInProgress = false
private[streaming] var rememberDuration: Duration = null
private[streaming] var checkpointInProgress = false
private[streaming] def start(time: Time) { var zeroTime: Time = null
var startTime: Time = null
var batchDuration: Duration = null
def start(time: Time) {
this.synchronized { this.synchronized {
if (zeroTime != null) { if (zeroTime != null) {
throw new Exception("DStream graph computation already started") throw new Exception("DStream graph computation already started")
} }
zeroTime = time zeroTime = time
startTime = time
outputStreams.foreach(_.initialize(zeroTime)) outputStreams.foreach(_.initialize(zeroTime))
outputStreams.foreach(_.remember(rememberDuration)) outputStreams.foreach(_.remember(rememberDuration))
outputStreams.foreach(_.validate) outputStreams.foreach(_.validate)
@ -29,19 +32,23 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
} }
} }
private[streaming] def stop() { def restart(time: Time) {
this.synchronized { startTime = time }
}
def stop() {
this.synchronized { this.synchronized {
inputStreams.par.foreach(_.stop()) inputStreams.par.foreach(_.stop())
} }
} }
private[streaming] def setContext(ssc: StreamingContext) { def setContext(ssc: StreamingContext) {
this.synchronized { this.synchronized {
outputStreams.foreach(_.setContext(ssc)) outputStreams.foreach(_.setContext(ssc))
} }
} }
private[streaming] def setBatchDuration(duration: Duration) { def setBatchDuration(duration: Duration) {
this.synchronized { this.synchronized {
if (batchDuration != null) { if (batchDuration != null) {
throw new Exception("Batch duration already set as " + batchDuration + throw new Exception("Batch duration already set as " + batchDuration +
@ -51,59 +58,68 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
batchDuration = duration batchDuration = duration
} }
private[streaming] def remember(duration: Duration) { def remember(duration: Duration) {
this.synchronized { this.synchronized {
if (rememberDuration != null) { if (rememberDuration != null) {
throw new Exception("Batch duration already set as " + batchDuration + throw new Exception("Batch duration already set as " + batchDuration +
". cannot set it again.") ". cannot set it again.")
} }
rememberDuration = duration
} }
rememberDuration = duration
} }
private[streaming] def addInputStream(inputStream: InputDStream[_]) { def addInputStream(inputStream: InputDStream[_]) {
this.synchronized { this.synchronized {
inputStream.setGraph(this) inputStream.setGraph(this)
inputStreams += inputStream inputStreams += inputStream
} }
} }
private[streaming] def addOutputStream(outputStream: DStream[_]) { def addOutputStream(outputStream: DStream[_]) {
this.synchronized { this.synchronized {
outputStream.setGraph(this) outputStream.setGraph(this)
outputStreams += outputStream outputStreams += outputStream
} }
} }
private[streaming] def getInputStreams() = this.synchronized { inputStreams.toArray } def getInputStreams() = this.synchronized { inputStreams.toArray }
private[streaming] def getOutputStreams() = this.synchronized { outputStreams.toArray } def getOutputStreams() = this.synchronized { outputStreams.toArray }
private[streaming] def generateRDDs(time: Time): Seq[Job] = { def generateJobs(time: Time): Seq[Job] = {
this.synchronized { this.synchronized {
outputStreams.flatMap(outputStream => outputStream.generateJob(time)) logInfo("Generating jobs for time " + time)
val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time))
logInfo("Generated " + jobs.length + " jobs for time " + time)
jobs
} }
} }
private[streaming] def forgetOldRDDs(time: Time) { def clearOldMetadata(time: Time) {
this.synchronized { this.synchronized {
outputStreams.foreach(_.forgetOldRDDs(time)) logInfo("Clearing old metadata for time " + time)
outputStreams.foreach(_.clearOldMetadata(time))
logInfo("Cleared old metadata for time " + time)
} }
} }
private[streaming] def updateCheckpointData(time: Time) { def updateCheckpointData(time: Time) {
this.synchronized { this.synchronized {
logInfo("Updating checkpoint data for time " + time)
outputStreams.foreach(_.updateCheckpointData(time)) outputStreams.foreach(_.updateCheckpointData(time))
logInfo("Updated checkpoint data for time " + time)
} }
} }
private[streaming] def restoreCheckpointData() { def restoreCheckpointData() {
this.synchronized { this.synchronized {
logInfo("Restoring checkpoint data")
outputStreams.foreach(_.restoreCheckpointData()) outputStreams.foreach(_.restoreCheckpointData())
logInfo("Restored checkpoint data")
} }
} }
private[streaming] def validate() { def validate() {
this.synchronized { this.synchronized {
assert(batchDuration != null, "Batch duration has not been set") assert(batchDuration != null, "Batch duration has not been set")
//assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low") //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low")

View file

@ -16,7 +16,7 @@ case class Duration (private val millis: Long) {
def * (times: Int): Duration = new Duration(millis * times) def * (times: Int): Duration = new Duration(millis * times)
def / (that: Duration): Long = millis / that.millis def / (that: Duration): Double = millis.toDouble / that.millis.toDouble
def isMultipleOf(that: Duration): Boolean = def isMultipleOf(that: Duration): Boolean =
(this.millis % that.millis == 0) (this.millis % that.millis == 0)

Some files were not shown because too many files have changed in this diff Show more