aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-01-20 12:48:15 -0800
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-01-20 12:48:15 -0800
commitfe85a075117a79675971aff0cd020bba446c0233 (patch)
tree652346ce2ef67ca10614e88c3d31cc2aadfcb9f0
parent17035db159e191a11cd86882c97078581073deb2 (diff)
parent86057ec7c868262763d1e31b3f3c94bd43eeafb3 (diff)
downloadspark-fe85a075117a79675971aff0cd020bba446c0233.tar.gz
spark-fe85a075117a79675971aff0cd020bba446c0233.tar.bz2
spark-fe85a075117a79675971aff0cd020bba446c0233.zip
Merge pull request #361 from mesos/streaming
Merge Streaming into master
-rw-r--r--.gitignore2
-rw-r--r--bagel/src/test/resources/log4j.properties4
-rw-r--r--core/src/main/scala/spark/CacheTracker.scala12
-rw-r--r--core/src/main/scala/spark/MapOutputTracker.scala28
-rw-r--r--core/src/main/scala/spark/PairRDDFunctions.scala31
-rw-r--r--core/src/main/scala/spark/ParallelCollection.scala27
-rw-r--r--core/src/main/scala/spark/RDD.scala171
-rw-r--r--core/src/main/scala/spark/RDDCheckpointData.scala105
-rw-r--r--core/src/main/scala/spark/SparkContext.scala83
-rw-r--r--core/src/main/scala/spark/Utils.scala2
-rw-r--r--core/src/main/scala/spark/api/java/JavaRDDLike.scala28
-rw-r--r--core/src/main/scala/spark/api/java/JavaSparkContext.scala26
-rw-r--r--core/src/main/scala/spark/api/python/PythonRDD.scala14
-rw-r--r--core/src/main/scala/spark/broadcast/HttpBroadcast.scala24
-rw-r--r--core/src/main/scala/spark/rdd/BlockRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/CartesianRDD.scala46
-rw-r--r--core/src/main/scala/spark/rdd/CheckpointRDD.scala128
-rw-r--r--core/src/main/scala/spark/rdd/CoGroupedRDD.scala52
-rw-r--r--core/src/main/scala/spark/rdd/CoalescedRDD.scala47
-rw-r--r--core/src/main/scala/spark/rdd/FilteredRDD.scala14
-rw-r--r--core/src/main/scala/spark/rdd/FlatMappedRDD.scala10
-rw-r--r--core/src/main/scala/spark/rdd/GlommedRDD.scala14
-rw-r--r--core/src/main/scala/spark/rdd/HadoopRDD.scala15
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsRDD.scala14
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala14
-rw-r--r--core/src/main/scala/spark/rdd/MappedRDD.scala11
-rw-r--r--core/src/main/scala/spark/rdd/NewHadoopRDD.scala13
-rw-r--r--core/src/main/scala/spark/rdd/PipedRDD.scala18
-rw-r--r--core/src/main/scala/spark/rdd/SampledRDD.scala28
-rw-r--r--core/src/main/scala/spark/rdd/ShuffledRDD.scala27
-rw-r--r--core/src/main/scala/spark/rdd/UnionRDD.scala44
-rw-r--r--core/src/main/scala/spark/rdd/ZippedRDD.scala59
-rw-r--r--core/src/main/scala/spark/scheduler/DAGScheduler.scala24
-rw-r--r--core/src/main/scala/spark/scheduler/ResultTask.scala94
-rw-r--r--core/src/main/scala/spark/scheduler/ShuffleMapTask.scala24
-rw-r--r--core/src/main/scala/spark/scheduler/local/LocalScheduler.scala8
-rw-r--r--core/src/main/scala/spark/storage/MemoryStore.scala1
-rw-r--r--core/src/main/scala/spark/util/MetadataCleaner.scala11
-rw-r--r--core/src/main/scala/spark/util/RateLimitedOutputStream.scala62
-rw-r--r--core/src/main/scala/spark/util/TimeStampedHashMap.scala24
-rw-r--r--core/src/main/scala/spark/util/TimeStampedHashSet.scala69
-rw-r--r--core/src/test/resources/log4j.properties4
-rw-r--r--core/src/test/scala/spark/CheckpointSuite.scala357
-rw-r--r--core/src/test/scala/spark/ClosureCleanerSuite.scala2
-rw-r--r--core/src/test/scala/spark/JavaAPISuite.java31
-rw-r--r--core/src/test/scala/spark/RDDSuite.scala29
-rw-r--r--core/src/test/scala/spark/scheduler/TaskContextSuite.scala5
-rw-r--r--core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala23
-rwxr-xr-xdocs/_layouts/global.html10
-rw-r--r--docs/_plugins/copy_api_dirs.rb4
-rw-r--r--docs/api.md5
-rw-r--r--docs/configuration.md11
-rw-r--r--docs/index.md6
-rw-r--r--docs/streaming-programming-guide.md313
-rw-r--r--examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala43
-rw-r--r--examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala36
-rw-r--r--examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java50
-rw-r--r--examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java62
-rw-r--r--examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java62
-rw-r--r--examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala69
-rw-r--r--examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala36
-rw-r--r--examples/src/main/scala/spark/streaming/examples/QueueStream.scala39
-rw-r--r--examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala46
-rw-r--r--examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala85
-rw-r--r--examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala84
-rw-r--r--examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala60
-rw-r--r--examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala71
-rw-r--r--project/SparkBuild.scala20
-rw-r--r--repl/src/test/resources/log4j.properties4
-rwxr-xr-xrun2
-rw-r--r--streaming/lib/kafka-0.7.2.jarbin0 -> 1358063 bytes
-rw-r--r--streaming/src/main/scala/spark/streaming/Checkpoint.scala118
-rw-r--r--streaming/src/main/scala/spark/streaming/DStream.scala657
-rw-r--r--streaming/src/main/scala/spark/streaming/DStreamGraph.scala134
-rw-r--r--streaming/src/main/scala/spark/streaming/Duration.scala62
-rw-r--r--streaming/src/main/scala/spark/streaming/Interval.scala41
-rw-r--r--streaming/src/main/scala/spark/streaming/Job.scala24
-rw-r--r--streaming/src/main/scala/spark/streaming/JobManager.scala33
-rw-r--r--streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala151
-rw-r--r--streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala562
-rw-r--r--streaming/src/main/scala/spark/streaming/Scheduler.scala77
-rw-r--r--streaming/src/main/scala/spark/streaming/StreamingContext.scala411
-rw-r--r--streaming/src/main/scala/spark/streaming/Time.scala42
-rw-r--r--streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala91
-rw-r--r--streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala183
-rw-r--r--streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala638
-rw-r--r--streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala346
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala40
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala19
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala102
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala21
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala20
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala20
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala137
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala28
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala17
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala19
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala200
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala21
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala21
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala20
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala254
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala41
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala90
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala149
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala27
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala103
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala84
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala19
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala40
-rw-r--r--streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala40
-rw-r--r--streaming/src/main/scala/spark/streaming/util/Clock.scala84
-rw-r--r--streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala98
-rw-r--r--streaming/src/main/scala/spark/streaming/util/RawTextSender.scala60
-rw-r--r--streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala75
-rw-r--r--streaming/src/test/java/JavaAPISuite.java1029
-rw-r--r--streaming/src/test/java/JavaTestUtils.scala65
-rw-r--r--streaming/src/test/resources/log4j.properties11
-rw-r--r--streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala218
-rw-r--r--streaming/src/test/scala/spark/streaming/CheckpointSuite.scala210
-rw-r--r--streaming/src/test/scala/spark/streaming/FailureSuite.scala191
-rw-r--r--streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala355
-rw-r--r--streaming/src/test/scala/spark/streaming/TestSuiteBase.scala291
-rw-r--r--streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala300
124 files changed, 10770 insertions, 332 deletions
diff --git a/.gitignore b/.gitignore
index c207409e3c..88d7b56181 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,6 +12,7 @@ third_party/libmesos.so
third_party/libmesos.dylib
conf/java-opts
conf/spark-env.sh
+conf/streaming-env.sh
conf/log4j.properties
docs/_site
docs/api
@@ -31,4 +32,5 @@ project/plugins/src_managed/
logs/
log/
spark-tests.log
+streaming-tests.log
dependency-reduced-pom.xml
diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
index 4c99e450bc..83d05cab2f 100644
--- a/bagel/src/test/resources/log4j.properties
+++ b/bagel/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the file bagel/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=bagel/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala
index 04c26b2e40..86ad737583 100644
--- a/core/src/main/scala/spark/CacheTracker.scala
+++ b/core/src/main/scala/spark/CacheTracker.scala
@@ -14,6 +14,7 @@ import akka.util.duration._
import spark.storage.BlockManager
import spark.storage.StorageLevel
+import util.{TimeStampedHashSet, MetadataCleaner, TimeStampedHashMap}
private[spark] sealed trait CacheTrackerMessage
@@ -30,7 +31,7 @@ private[spark] case object StopCacheTracker extends CacheTrackerMessage
private[spark] class CacheTrackerActor extends Actor with Logging {
// TODO: Should probably store (String, CacheType) tuples
- private val locs = new HashMap[Int, Array[List[String]]]
+ private val locs = new TimeStampedHashMap[Int, Array[List[String]]]
/**
* A map from the slave's host name to its cache size.
@@ -38,6 +39,8 @@ private[spark] class CacheTrackerActor extends Actor with Logging {
private val slaveCapacity = new HashMap[String, Long]
private val slaveUsage = new HashMap[String, Long]
+ private val metadataCleaner = new MetadataCleaner("CacheTrackerActor", locs.clearOldValues)
+
private def getCacheUsage(host: String): Long = slaveUsage.getOrElse(host, 0L)
private def getCacheCapacity(host: String): Long = slaveCapacity.getOrElse(host, 0L)
private def getCacheAvailable(host: String): Long = getCacheCapacity(host) - getCacheUsage(host)
@@ -86,6 +89,7 @@ private[spark] class CacheTrackerActor extends Actor with Logging {
case StopCacheTracker =>
logInfo("Stopping CacheTrackerActor")
sender ! true
+ metadataCleaner.cancel()
context.stop(self)
}
}
@@ -109,11 +113,15 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b
actorSystem.actorFor(url)
}
- val registeredRddIds = new HashSet[Int]
+ // TODO: Consider removing this HashSet completely as locs CacheTrackerActor already
+ // keeps track of registered RDDs
+ val registeredRddIds = new TimeStampedHashSet[Int]
// Remembers which splits are currently being loaded (on worker nodes)
val loading = new HashSet[String]
+ val metadataCleaner = new MetadataCleaner("CacheTracker", registeredRddIds.clearOldValues)
+
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
def askTracker(message: Any): Any = {
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index 9f2aa76830..ac02f3363a 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -17,6 +17,7 @@ import akka.util.duration._
import spark.scheduler.MapStatus
import spark.storage.BlockManagerId
+import spark.util.{MetadataCleaner, TimeStampedHashMap}
private[spark] sealed trait MapOutputTrackerMessage
@@ -44,7 +45,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
val timeout = 10.seconds
- var mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
+ var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
// Incremented every time a fetch fails so that client nodes know to clear
// their cache of map output locations if this happens.
@@ -53,7 +54,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
// Cache a serialized version of the output statuses for each shuffle to send them out faster
var cacheGeneration = generation
- val cachedSerializedStatuses = new HashMap[Int, Array[Byte]]
+ val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
var trackerActor: ActorRef = if (isMaster) {
val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName)
@@ -64,6 +65,8 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
actorSystem.actorFor(url)
}
+ val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
+
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
def askTracker(message: Any): Any = {
@@ -84,14 +87,14 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
def registerShuffle(shuffleId: Int, numMaps: Int) {
- if (mapStatuses.get(shuffleId) != null) {
+ if (mapStatuses.get(shuffleId) != None) {
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
}
mapStatuses.put(shuffleId, new Array[MapStatus](numMaps))
}
def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
- var array = mapStatuses.get(shuffleId)
+ var array = mapStatuses(shuffleId)
array.synchronized {
array(mapId) = status
}
@@ -108,7 +111,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
- var array = mapStatuses.get(shuffleId)
+ var array = mapStatuses(shuffleId)
if (array != null) {
array.synchronized {
if (array(mapId) != null && array(mapId).address == bmAddress) {
@@ -126,7 +129,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
// Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle
def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = {
- val statuses = mapStatuses.get(shuffleId)
+ val statuses = mapStatuses.get(shuffleId).orNull
if (statuses == null) {
logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
fetching.synchronized {
@@ -139,8 +142,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
case e: InterruptedException =>
}
}
- return MapOutputTracker.convertMapStatuses(shuffleId, reduceId,
- mapStatuses.get(shuffleId))
+ return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, mapStatuses(shuffleId))
} else {
fetching += shuffleId
}
@@ -168,9 +170,15 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
}
+ def cleanup(cleanupTime: Long) {
+ mapStatuses.clearOldValues(cleanupTime)
+ cachedSerializedStatuses.clearOldValues(cleanupTime)
+ }
+
def stop() {
communicate(StopMapOutputTracker)
mapStatuses.clear()
+ metadataCleaner.cancel()
trackerActor = null
}
@@ -196,7 +204,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
generationLock.synchronized {
if (newGen > generation) {
logInfo("Updating generation to " + newGen + " and clearing cache")
- mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
+ mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
generation = newGen
}
}
@@ -214,7 +222,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
case Some(bytes) =>
return bytes
case None =>
- statuses = mapStatuses.get(shuffleId)
+ statuses = mapStatuses(shuffleId)
generationGotten = generation
}
}
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index ce48cea903..abb01c387c 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -199,9 +199,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
/**
- * 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
- * "combiner" in MapReduce.
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
*/
def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
this.cogroup(other, partitioner).flatMapValues {
@@ -661,24 +661,23 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
}
private[spark]
-class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override val partitioner = prev.partitioner
- override def compute(split: Split, taskContext: TaskContext) =
- prev.iterator(split, taskContext).map{case (k, v) => (k, f(v))}
+class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U)
+ extends RDD[(K, U)](prev) {
+
+ override def getSplits = firstParent[(K, V)].splits
+ override val partitioner = firstParent[(K, V)].partitioner
+ override def compute(split: Split, context: TaskContext) =
+ firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) }
}
private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
- extends RDD[(K, U)](prev.context) {
-
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override val partitioner = prev.partitioner
+ extends RDD[(K, U)](prev) {
- override def compute(split: Split, taskContext: TaskContext) = {
- prev.iterator(split, taskContext).flatMap { case (k, v) => f(v).map(x => (k, x)) }
+ override def getSplits = firstParent[(K, V)].splits
+ override val partitioner = firstParent[(K, V)].partitioner
+ override def compute(split: Split, context: TaskContext) = {
+ firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) }
}
}
diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala
index a27f766e31..ede933c9e9 100644
--- a/core/src/main/scala/spark/ParallelCollection.scala
+++ b/core/src/main/scala/spark/ParallelCollection.scala
@@ -2,6 +2,7 @@ package spark
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
+import scala.collection.Map
private[spark] class ParallelCollectionSplit[T: ClassManifest](
val rddId: Long,
@@ -22,30 +23,40 @@ private[spark] class ParallelCollectionSplit[T: ClassManifest](
}
private[spark] class ParallelCollection[T: ClassManifest](
- sc: SparkContext,
+ @transient sc : SparkContext,
@transient data: Seq[T],
- numSlices: Int)
- extends RDD[T](sc) {
+ numSlices: Int,
+ locationPrefs : Map[Int,Seq[String]])
+ extends RDD[T](sc, Nil) {
// TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets
// cached. It might be worthwhile to write the data to a file in the DFS and read it in the split
// instead.
+ // UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal.
@transient
- val splits_ = {
+ var splits_ : Array[Split] = {
val slices = ParallelCollection.slice(data, numSlices).toArray
slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray
}
- override def splits = splits_.asInstanceOf[Array[Split]]
+ override def getSplits = splits_.asInstanceOf[Array[Split]]
- override def compute(s: Split, taskContext: TaskContext) =
+ override def compute(s: Split, context: TaskContext) =
s.asInstanceOf[ParallelCollectionSplit[T]].iterator
- override def preferredLocations(s: Split): Seq[String] = Nil
+ override def getPreferredLocations(s: Split): Seq[String] = {
+ locationPrefs.get(s.index) match {
+ case Some(s) => s
+ case _ => Nil
+ }
+ }
- override val dependencies: List[Dependency[_]] = Nil
+ override def clearDependencies() {
+ splits_ = null
+ }
}
+
private object ParallelCollection {
/**
* Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index 3b9ced1946..e0d2eabb1d 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -1,10 +1,8 @@
package spark
-import java.io.EOFException
-import java.io.ObjectInputStream
+import java.io.{ObjectOutputStream, IOException, EOFException, ObjectInputStream}
import java.net.URL
-import java.util.Random
-import java.util.Date
+import java.util.{Date, Random}
import java.util.{HashMap => JHashMap}
import java.util.concurrent.atomic.AtomicLong
@@ -13,6 +11,7 @@ import scala.collection.JavaConversions.mapAsScalaMap
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
+import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.BytesWritable
import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.io.Text
@@ -73,41 +72,42 @@ import SparkContext._
* [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
* on RDD internals.
*/
-abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serializable {
+abstract class RDD[T: ClassManifest](
+ @transient var sc: SparkContext,
+ var dependencies_ : List[Dependency[_]]
+ ) extends Serializable with Logging {
- // Methods that must be implemented by subclasses:
- /** Set of partitions in this RDD. */
- def splits: Array[Split]
+ def this(@transient oneParent: RDD[_]) =
+ this(oneParent.context , List(new OneToOneDependency(oneParent)))
+
+ // =======================================================================
+ // Methods that should be implemented by subclasses of RDD
+ // =======================================================================
/** Function for computing a given partition. */
def compute(split: Split, context: TaskContext): Iterator[T]
- /** How this RDD depends on any parent RDDs. */
- @transient val dependencies: List[Dependency[_]]
+ /** Set of partitions in this RDD. */
+ protected def getSplits(): Array[Split]
- // Methods available on all RDDs:
+ /** How this RDD depends on any parent RDDs. */
+ protected def getDependencies(): List[Dependency[_]] = dependencies_
- /** Record user function generating this RDD. */
- private[spark] val origin = Utils.getSparkCallSite
+ /** Optionally overridden by subclasses to specify placement preferences. */
+ protected def getPreferredLocations(split: Split): Seq[String] = Nil
/** Optionally overridden by subclasses to specify how they are partitioned. */
val partitioner: Option[Partitioner] = None
- /** Optionally overridden by subclasses to specify placement preferences. */
- def preferredLocations(split: Split): Seq[String] = Nil
- /** The [[spark.SparkContext]] that this RDD was created on. */
- def context = sc
-
- private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+ // =======================================================================
+ // Methods and fields available on all RDDs
+ // =======================================================================
/** A unique ID for this RDD (within its SparkContext). */
val id = sc.newRddId()
- // Variables relating to persistence
- private var storageLevel: StorageLevel = StorageLevel.NONE
-
/**
* Set this RDD's storage level to persist its values across operations after the first time
* it is computed. Can only be called once on each RDD.
@@ -131,22 +131,39 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
/** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
def getStorageLevel = storageLevel
- private[spark] def checkpoint(level: StorageLevel = StorageLevel.MEMORY_AND_DISK_2): RDD[T] = {
- if (!level.useDisk && level.replication < 2) {
- throw new Exception("Cannot checkpoint without using disk or replication (level requested was " + level + ")")
+ /**
+ * Get the preferred location of a split, taking into account whether the
+ * RDD is checkpointed or not.
+ */
+ final def preferredLocations(split: Split): Seq[String] = {
+ if (isCheckpointed) {
+ checkpointData.get.getPreferredLocations(split)
+ } else {
+ getPreferredLocations(split)
}
+ }
- // This is a hack. Ideally this should re-use the code used by the CacheTracker
- // to generate the key.
- def getSplitKey(split: Split) = "rdd_%d_%d".format(this.id, split.index)
-
- persist(level)
- sc.runJob(this, (iter: Iterator[T]) => {} )
-
- val p = this.partitioner
+ /**
+ * Get the array of splits of this RDD, taking into account whether the
+ * RDD is checkpointed or not.
+ */
+ final def splits: Array[Split] = {
+ if (isCheckpointed) {
+ checkpointData.get.getSplits
+ } else {
+ getSplits
+ }
+ }
- new BlockRDD[T](sc, splits.map(getSplitKey).toArray) {
- override val partitioner = p
+ /**
+ * Get the list of dependencies of this RDD, taking into account whether the
+ * RDD is checkpointed or not.
+ */
+ final def dependencies: List[Dependency[_]] = {
+ if (isCheckpointed) {
+ dependencies_
+ } else {
+ getDependencies
}
}
@@ -156,7 +173,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
* subclasses of RDD.
*/
final def iterator(split: Split, context: TaskContext): Iterator[T] = {
- if (storageLevel != StorageLevel.NONE) {
+ if (isCheckpointed) {
+ checkpointData.get.iterator(split, context)
+ } else if (storageLevel != StorageLevel.NONE) {
SparkEnv.get.cacheTracker.getOrCompute[T](this, split, context, storageLevel)
} else {
compute(split, context)
@@ -528,4 +547,84 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
private[spark] def collectPartitions(): Array[Array[T]] = {
sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
}
+
+ /**
+ * Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint
+ * directory set with SparkContext.setCheckpointDir() and all references to its parent
+ * RDDs will be removed. This function must be called before any job has been
+ * executed on this RDD. It is strongly recommended that this RDD is persisted in
+ * memory, otherwise saving it on a file will require recomputation.
+ */
+ def checkpoint() {
+ if (context.checkpointDir.isEmpty) {
+ throw new Exception("Checkpoint directory has not been set in the SparkContext")
+ } else if (checkpointData.isEmpty) {
+ checkpointData = Some(new RDDCheckpointData(this))
+ checkpointData.get.markForCheckpoint()
+ }
+ }
+
+ /**
+ * Return whether this RDD has been checkpointed or not
+ */
+ def isCheckpointed(): Boolean = {
+ if (checkpointData.isDefined) checkpointData.get.isCheckpointed() else false
+ }
+
+ /**
+ * Gets the name of the file to which this RDD was checkpointed
+ */
+ def getCheckpointFile(): Option[String] = {
+ if (checkpointData.isDefined) checkpointData.get.getCheckpointFile() else None
+ }
+
+ // =======================================================================
+ // Other internal methods and fields
+ // =======================================================================
+
+ private var storageLevel: StorageLevel = StorageLevel.NONE
+
+ /** Record user function generating this RDD. */
+ private[spark] val origin = Utils.getSparkCallSite
+
+ private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+
+ private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
+
+ /** Returns the first parent RDD */
+ protected[spark] def firstParent[U: ClassManifest] = {
+ dependencies.head.rdd.asInstanceOf[RDD[U]]
+ }
+
+ /** The [[spark.SparkContext]] that this RDD was created on. */
+ def context = sc
+
+ /**
+ * Performs the checkpointing of this RDD by saving this . It is called by the DAGScheduler
+ * after a job using this RDD has completed (therefore the RDD has been materialized and
+ * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs.
+ */
+ protected[spark] def doCheckpoint() {
+ if (checkpointData.isDefined) checkpointData.get.doCheckpoint()
+ dependencies.foreach(_.rdd.doCheckpoint())
+ }
+
+ /**
+ * Changes the dependencies of this RDD from its original parents to the new RDD
+ * (`newRDD`) created from the checkpoint file.
+ */
+ protected[spark] def changeDependencies(newRDD: RDD[_]) {
+ clearDependencies()
+ dependencies_ = List(new OneToOneDependency(newRDD))
+ }
+
+ /**
+ * Clears the dependencies of this RDD. This method must ensure that all references
+ * to the original parent RDDs is removed to enable the parent RDDs to be garbage
+ * collected. Subclasses of RDD may override this method for implementing their own cleaning
+ * logic. See [[spark.rdd.UnionRDD]] and [[spark.rdd.ShuffledRDD]] to get a better idea.
+ */
+ protected[spark] def clearDependencies() {
+ dependencies_ = null
+ }
}
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala
new file mode 100644
index 0000000000..18df530b7d
--- /dev/null
+++ b/core/src/main/scala/spark/RDDCheckpointData.scala
@@ -0,0 +1,105 @@
+package spark
+
+import org.apache.hadoop.fs.Path
+import rdd.{CheckpointRDD, CoalescedRDD}
+import scheduler.{ResultTask, ShuffleMapTask}
+
+/**
+ * Enumeration to manage state transitions of an RDD through checkpointing
+ * [ Initialized --> marked for checkpointing --> checkpointing in progress --> checkpointed ]
+ */
+private[spark] object CheckpointState extends Enumeration {
+ type CheckpointState = Value
+ val Initialized, MarkedForCheckpoint, CheckpointingInProgress, Checkpointed = Value
+}
+
+/**
+ * This class contains all the information related to RDD checkpointing. Each instance of this class
+ * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as,
+ * manages the post-checkpoint state by providing the updated splits, iterator and preferred locations
+ * of the checkpointed RDD.
+ */
+private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
+extends Logging with Serializable {
+
+ import CheckpointState._
+
+ // The checkpoint state of the associated RDD.
+ var cpState = Initialized
+
+ // The file to which the associated RDD has been checkpointed to
+ @transient var cpFile: Option[String] = None
+
+ // The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD.
+ @transient var cpRDD: Option[RDD[T]] = None
+
+ // Mark the RDD for checkpointing
+ def markForCheckpoint() {
+ RDDCheckpointData.synchronized {
+ if (cpState == Initialized) cpState = MarkedForCheckpoint
+ }
+ }
+
+ // Is the RDD already checkpointed
+ def isCheckpointed(): Boolean = {
+ RDDCheckpointData.synchronized { cpState == Checkpointed }
+ }
+
+ // Get the file to which this RDD was checkpointed to as an Option
+ def getCheckpointFile(): Option[String] = {
+ RDDCheckpointData.synchronized { cpFile }
+ }
+
+ // Do the checkpointing of the RDD. Called after the first job using that RDD is over.
+ def doCheckpoint() {
+ // If it is marked for checkpointing AND checkpointing is not already in progress,
+ // then set it to be in progress, else return
+ RDDCheckpointData.synchronized {
+ if (cpState == MarkedForCheckpoint) {
+ cpState = CheckpointingInProgress
+ } else {
+ return
+ }
+ }
+
+ // Save to file, and reload it as an RDD
+ val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id).toString
+ rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _)
+ val newRDD = new CheckpointRDD[T](rdd.context, path)
+
+ // Change the dependencies and splits of the RDD
+ RDDCheckpointData.synchronized {
+ cpFile = Some(path)
+ cpRDD = Some(newRDD)
+ rdd.changeDependencies(newRDD)
+ cpState = Checkpointed
+ RDDCheckpointData.clearTaskCaches()
+ logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id)
+ }
+ }
+
+ // Get preferred location of a split after checkpointing
+ def getPreferredLocations(split: Split) = {
+ RDDCheckpointData.synchronized {
+ cpRDD.get.preferredLocations(split)
+ }
+ }
+
+ def getSplits: Array[Split] = {
+ RDDCheckpointData.synchronized {
+ cpRDD.get.splits
+ }
+ }
+
+ // Get iterator. This is called at the worker nodes.
+ def iterator(split: Split, context: TaskContext): Iterator[T] = {
+ rdd.firstParent[T].iterator(split, context)
+ }
+}
+
+private[spark] object RDDCheckpointData {
+ def clearTaskCaches() {
+ ShuffleMapTask.clearCache()
+ ResultTask.clearCache()
+ }
+}
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index bbf8272eb3..7f3259d982 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -3,6 +3,7 @@ package spark
import java.io._
import java.util.concurrent.atomic.AtomicInteger
import java.net.{URI, URLClassLoader}
+import java.lang.ref.WeakReference
import scala.collection.Map
import scala.collection.generic.Growable
@@ -36,12 +37,8 @@ import spark.broadcast._
import spark.deploy.LocalSparkCluster
import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
-import spark.rdd.HadoopRDD
-import spark.rdd.NewHadoopRDD
-import spark.rdd.UnionRDD
-import spark.scheduler.ShuffleMapTask
-import spark.scheduler.DAGScheduler
-import spark.scheduler.TaskScheduler
+import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD}
+import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler}
import spark.scheduler.local.LocalScheduler
import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
@@ -58,10 +55,10 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend
* @param environment Environment variables to set on worker nodes.
*/
class SparkContext(
- master: String,
- jobName: String,
+ val master: String,
+ val jobName: String,
val sparkHome: String,
- jars: Seq[String],
+ val jars: Seq[String],
environment: Map[String, String])
extends Logging {
@@ -187,11 +184,13 @@ class SparkContext(
private var dagScheduler = new DAGScheduler(taskScheduler)
+ private[spark] var checkpointDir: Option[String] = None
+
// Methods for creating RDDs
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
- new ParallelCollection[T](this, seq, numSlices)
+ new ParallelCollection[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
/** Distribute a local Scala collection to form an RDD. */
@@ -199,6 +198,14 @@ class SparkContext(
parallelize(seq, numSlices)
}
+ /** Distribute a local Scala collection to form an RDD, with one or more
+ * location preferences (hostnames of Spark nodes) for each object.
+ * Create a new partition for each collection item. */
+ def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
+ val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
+ new ParallelCollection[T](this, seq.map(_._1), seq.size, indexToPrefs)
+ }
+
/**
* Read a text file from HDFS, a local file system (available on all nodes), or any
* Hadoop-supported file system URI, and return it as an RDD of Strings.
@@ -365,6 +372,13 @@ class SparkContext(
.flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes))
}
+
+ protected[spark] def checkpointFile[T: ClassManifest](
+ path: String
+ ): RDD[T] = {
+ new CheckpointRDD[T](this, path)
+ }
+
/** Build the union of a list of RDDs. */
def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
@@ -471,17 +485,22 @@ class SparkContext(
/** Shut down the SparkContext. */
def stop() {
- dagScheduler.stop()
- dagScheduler = null
- taskScheduler = null
- // TODO: Cache.stop()?
- env.stop()
- // Clean up locally linked files
- clearFiles()
- clearJars()
- SparkEnv.set(null)
- ShuffleMapTask.clearCache()
- logInfo("Successfully stopped SparkContext")
+ if (dagScheduler != null) {
+ dagScheduler.stop()
+ dagScheduler = null
+ taskScheduler = null
+ // TODO: Cache.stop()?
+ env.stop()
+ // Clean up locally linked files
+ clearFiles()
+ clearJars()
+ SparkEnv.set(null)
+ ShuffleMapTask.clearCache()
+ ResultTask.clearCache()
+ logInfo("Successfully stopped SparkContext")
+ } else {
+ logInfo("SparkContext already stopped")
+ }
}
/**
@@ -518,6 +537,7 @@ class SparkContext(
val start = System.nanoTime
val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal)
logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
+ rdd.doCheckpoint()
result
}
@@ -574,6 +594,26 @@ class SparkContext(
return f
}
+ /**
+ * Set the directory under which RDDs are going to be checkpointed. The directory must
+ * be a HDFS path if running on a cluster. If the directory does not exist, it will
+ * be created. If the directory exists and useExisting is set to true, then the
+ * exisiting directory will be used. Otherwise an exception will be thrown to
+ * prevent accidental overriding of checkpoint files in the existing directory.
+ */
+ def setCheckpointDir(dir: String, useExisting: Boolean = false) {
+ val path = new Path(dir)
+ val fs = path.getFileSystem(new Configuration())
+ if (!useExisting) {
+ if (fs.exists(path)) {
+ throw new Exception("Checkpoint directory '" + path + "' already exists.")
+ } else {
+ fs.mkdirs(path)
+ }
+ }
+ checkpointDir = Some(dir)
+ }
+
/** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */
def defaultParallelism: Int = taskScheduler.defaultParallelism
@@ -595,6 +635,7 @@ class SparkContext(
* various Spark features.
*/
object SparkContext {
+
implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
def addInPlace(t1: Double, t2: Double): Double = t1 + t2
def zero(initialValue: Double) = 0.0
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index aeed5d2f32..b3421df27c 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -315,7 +315,7 @@ private object Utils extends Logging {
* millisecond.
*/
def getUsedTimeMs(startTimeMs: Long): String = {
- return " " + (System.currentTimeMillis - startTimeMs) + " ms "
+ return " " + (System.currentTimeMillis - startTimeMs) + " ms"
}
/**
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index d15f6dd02f..087270e46d 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -9,6 +9,7 @@ import spark.api.java.JavaPairRDD._
import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
import spark.partial.{PartialResult, BoundedDouble}
import spark.storage.StorageLevel
+import com.google.common.base.Optional
trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
@@ -306,4 +307,31 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
JavaPairRDD.fromRDD(rdd.keyBy(f))
}
+
+ /**
+ * Mark this RDD for checkpointing. The RDD will be saved to a file inside `checkpointDir`
+ * (set using setCheckpointDir()) and all references to its parent RDDs will be removed.
+ * This is used to truncate very long lineages. In the current implementation, Spark will save
+ * this RDD to a file (using saveAsObjectFile()) after the first job using this RDD is done.
+ * Hence, it is strongly recommended to use checkpoint() on RDDs when
+ * (i) checkpoint() is called before the any job has been executed on this RDD.
+ * (ii) This RDD has been made to persist in memory. Otherwise saving it on a file will
+ * require recomputation.
+ */
+ def checkpoint() = rdd.checkpoint()
+
+ /**
+ * Return whether this RDD has been checkpointed or not
+ */
+ def isCheckpointed(): Boolean = rdd.isCheckpointed()
+
+ /**
+ * Gets the name of the file to which this RDD was checkpointed
+ */
+ def getCheckpointFile(): Optional[String] = {
+ rdd.getCheckpointFile match {
+ case Some(file) => Optional.of(file)
+ case _ => Optional.absent()
+ }
+ }
}
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index 88ab2846be..fa2f14113d 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -355,6 +355,32 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
def clearFiles() {
sc.clearFiles()
}
+
+ /**
+ * Set the directory under which RDDs are going to be checkpointed. This method will
+ * create this directory and will throw an exception of the path already exists (to avoid
+ * overwriting existing files may be overwritten). The directory will be deleted on exit
+ * if indicated.
+ */
+ def setCheckpointDir(dir: String, useExisting: Boolean) {
+ sc.setCheckpointDir(dir, useExisting)
+ }
+
+ /**
+ * Set the directory under which RDDs are going to be checkpointed. This method will
+ * create this directory and will throw an exception of the path already exists (to avoid
+ * overwriting existing files may be overwritten). The directory will be deleted on exit
+ * if indicated.
+ */
+ def setCheckpointDir(dir: String) {
+ sc.setCheckpointDir(dir)
+ }
+
+ protected def checkpointFile[T](path: String): JavaRDD[T] = {
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ new JavaRDD(sc.checkpointFile(path))
+ }
}
object JavaSparkContext {
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index fb13e84658..89f7c316dc 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -21,7 +21,7 @@ private[spark] class PythonRDD[T: ClassManifest](
pythonExec: String,
broadcastVars: JList[Broadcast[Array[Byte]]],
accumulator: Accumulator[JList[Array[Byte]]])
- extends RDD[Array[Byte]](parent.context) {
+ extends RDD[Array[Byte]](parent) {
// Similar to Runtime.exec(), if we are given a single string, split it into words
// using a standard StringTokenizer (i.e. by spaces)
@@ -32,9 +32,7 @@ private[spark] class PythonRDD[T: ClassManifest](
this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec,
broadcastVars, accumulator)
- override def splits = parent.splits
-
- override val dependencies = List(new OneToOneDependency(parent))
+ override def getSplits = parent.splits
override val partitioner = if (preservePartitoning) parent.partitioner else None
@@ -137,6 +135,8 @@ private[spark] class PythonRDD[T: ClassManifest](
}
}
+ override def checkpoint() { }
+
val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
}
@@ -145,14 +145,14 @@ private[spark] class PythonRDD[T: ClassManifest](
* This is used by PySpark's shuffle operations.
*/
private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
- RDD[(Array[Byte], Array[Byte])](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
+ RDD[(Array[Byte], Array[Byte])](prev) {
+ override def getSplits = prev.splits
override def compute(split: Split, context: TaskContext) =
prev.iterator(split, context).grouped(2).map {
case Seq(a, b) => (a, b)
case x => throw new Exception("PairwiseRDD: unexpected value: " + x)
}
+ override def checkpoint() { }
val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this)
}
diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
index 856a4683a9..8e490e6bad 100644
--- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
@@ -11,6 +11,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
import spark._
import spark.storage.StorageLevel
+import util.{MetadataCleaner, TimeStampedHashSet}
private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
extends Broadcast[T](id) with Logging with Serializable {
@@ -64,6 +65,10 @@ private object HttpBroadcast extends Logging {
private var serverUri: String = null
private var server: HttpServer = null
+ private val files = new TimeStampedHashSet[String]
+ private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup)
+
+
def initialize(isMaster: Boolean) {
synchronized {
if (!initialized) {
@@ -85,6 +90,7 @@ private object HttpBroadcast extends Logging {
server = null
}
initialized = false
+ cleaner.cancel()
}
}
@@ -108,6 +114,7 @@ private object HttpBroadcast extends Logging {
val serOut = ser.serializeStream(out)
serOut.writeObject(value)
serOut.close()
+ files += file.getAbsolutePath
}
def read[T](id: Long): T = {
@@ -123,4 +130,21 @@ private object HttpBroadcast extends Logging {
serIn.close()
obj
}
+
+ def cleanup(cleanupTime: Long) {
+ val iterator = files.internalMap.entrySet().iterator()
+ while(iterator.hasNext) {
+ val entry = iterator.next()
+ val (file, time) = (entry.getKey, entry.getValue)
+ if (time < cleanupTime) {
+ try {
+ iterator.remove()
+ new File(file.toString).delete()
+ logInfo("Deleted broadcast file '" + file + "'")
+ } catch {
+ case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e)
+ }
+ }
+ }
+ }
}
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index f98528a183..b1095a52b4 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,9 +1,7 @@
package spark.rdd
import scala.collection.mutable.HashMap
-
-import spark.{Dependency, RDD, SparkContext, SparkEnv, Split, TaskContext}
-
+import spark.{RDD, SparkContext, SparkEnv, Split, TaskContext}
private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
val index = idx
@@ -11,10 +9,10 @@ private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split
private[spark]
class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
- extends RDD[T](sc) {
+ extends RDD[T](sc, Nil) {
@transient
- val splits_ = (0 until blockIds.size).map(i => {
+ var splits_ : Array[Split] = (0 until blockIds.size).map(i => {
new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split]
}).toArray
@@ -26,7 +24,7 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
HashMap(blockIds.zip(locations):_*)
}
- override def splits = splits_
+ override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[T] = {
val blockManager = SparkEnv.get.blockManager
@@ -38,9 +36,11 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
}
}
- override def preferredLocations(split: Split) =
+ override def getPreferredLocations(split: Split) =
locations_(split.asInstanceOf[BlockRDDSplit].blockId)
- override val dependencies: List[Dependency[_]] = Nil
+ override def clearDependencies() {
+ splits_ = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala
index 4a7e5f3d06..79e7c24e7c 100644
--- a/core/src/main/scala/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala
@@ -1,37 +1,54 @@
package spark.rdd
-import spark.{NarrowDependency, RDD, SparkContext, Split, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
+import spark.{OneToOneDependency, NarrowDependency, RDD, SparkContext, Split, TaskContext}
private[spark]
-class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
+class CartesianSplit(
+ idx: Int,
+ @transient rdd1: RDD[_],
+ @transient rdd2: RDD[_],
+ s1Index: Int,
+ s2Index: Int
+ ) extends Split {
+ var s1 = rdd1.splits(s1Index)
+ var s2 = rdd2.splits(s2Index)
override val index: Int = idx
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent split at the time of task serialization
+ s1 = rdd1.splits(s1Index)
+ s2 = rdd2.splits(s2Index)
+ oos.defaultWriteObject()
+ }
}
private[spark]
class CartesianRDD[T: ClassManifest, U:ClassManifest](
sc: SparkContext,
- rdd1: RDD[T],
- rdd2: RDD[U])
- extends RDD[Pair[T, U]](sc)
+ var rdd1 : RDD[T],
+ var rdd2 : RDD[U])
+ extends RDD[Pair[T, U]](sc, Nil)
with Serializable {
val numSplitsInRdd2 = rdd2.splits.size
@transient
- val splits_ = {
+ var splits_ = {
// create the cross product split
val array = new Array[Split](rdd1.splits.size * rdd2.splits.size)
for (s1 <- rdd1.splits; s2 <- rdd2.splits) {
val idx = s1.index * numSplitsInRdd2 + s2.index
- array(idx) = new CartesianSplit(idx, s1, s2)
+ array(idx) = new CartesianSplit(idx, rdd1, rdd2, s1.index, s2.index)
}
array
}
- override def splits = splits_
+ override def getSplits = splits_
- override def preferredLocations(split: Split) = {
+ override def getPreferredLocations(split: Split) = {
val currSplit = split.asInstanceOf[CartesianSplit]
rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)
}
@@ -42,7 +59,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
}
- override val dependencies = List(
+ var deps_ = List(
new NarrowDependency(rdd1) {
def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2)
},
@@ -50,4 +67,13 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2)
}
)
+
+ override def getDependencies = deps_
+
+ override def clearDependencies() {
+ deps_ = Nil
+ splits_ = null
+ rdd1 = null
+ rdd2 = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
new file mode 100644
index 0000000000..6f00f6ac73
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -0,0 +1,128 @@
+package spark.rdd
+
+import spark._
+import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.{NullWritable, BytesWritable}
+import org.apache.hadoop.util.ReflectionUtils
+import org.apache.hadoop.fs.Path
+import java.io.{File, IOException, EOFException}
+import java.text.NumberFormat
+
+private[spark] class CheckpointRDDSplit(idx: Int, val splitFile: String) extends Split {
+ override val index: Int = idx
+}
+
+/**
+ * This RDD represents a RDD checkpoint file (similar to HadoopRDD).
+ */
+private[spark]
+class CheckpointRDD[T: ClassManifest](sc: SparkContext, checkpointPath: String)
+ extends RDD[T](sc, Nil) {
+
+ @transient val path = new Path(checkpointPath)
+ @transient val fs = path.getFileSystem(new Configuration())
+
+ @transient val splits_ : Array[Split] = {
+ val splitFiles = fs.listStatus(path).map(_.getPath.toString).filter(_.contains("part-")).sorted
+ splitFiles.zipWithIndex.map(x => new CheckpointRDDSplit(x._2, x._1)).toArray
+ }
+
+ checkpointData = Some(new RDDCheckpointData[T](this))
+ checkpointData.get.cpFile = Some(checkpointPath)
+
+ override def getSplits = splits_
+
+ override def getPreferredLocations(split: Split): Seq[String] = {
+ val status = fs.getFileStatus(path)
+ val locations = fs.getFileBlockLocations(status, 0, status.getLen)
+ locations.firstOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
+ }
+
+ override def compute(split: Split, context: TaskContext): Iterator[T] = {
+ CheckpointRDD.readFromFile(split.asInstanceOf[CheckpointRDDSplit].splitFile, context)
+ }
+
+ override def checkpoint() {
+ // Do nothing. Hadoop RDD should not be checkpointed.
+ }
+}
+
+private[spark] object CheckpointRDD extends Logging {
+
+ def splitIdToFileName(splitId: Int): String = {
+ val numfmt = NumberFormat.getInstance()
+ numfmt.setMinimumIntegerDigits(5)
+ numfmt.setGroupingUsed(false)
+ "part-" + numfmt.format(splitId)
+ }
+
+ def writeToFile[T](path: String, blockSize: Int = -1)(context: TaskContext, iterator: Iterator[T]) {
+ val outputDir = new Path(path)
+ val fs = outputDir.getFileSystem(new Configuration())
+
+ val finalOutputName = splitIdToFileName(context.splitId)
+ val finalOutputPath = new Path(outputDir, finalOutputName)
+ val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + context.attemptId)
+
+ if (fs.exists(tempOutputPath)) {
+ throw new IOException("Checkpoint failed: temporary path " +
+ tempOutputPath + " already exists")
+ }
+ val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+
+ val fileOutputStream = if (blockSize < 0) {
+ fs.create(tempOutputPath, false, bufferSize)
+ } else {
+ // This is mainly for testing purpose
+ fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize)
+ }
+ val serializer = SparkEnv.get.serializer.newInstance()
+ val serializeStream = serializer.serializeStream(fileOutputStream)
+ serializeStream.writeAll(iterator)
+ serializeStream.close()
+
+ if (!fs.rename(tempOutputPath, finalOutputPath)) {
+ if (!fs.delete(finalOutputPath, true)) {
+ throw new IOException("Checkpoint failed: failed to delete earlier output of task "
+ + context.attemptId)
+ }
+ if (!fs.rename(tempOutputPath, finalOutputPath)) {
+ throw new IOException("Checkpoint failed: failed to save output of task: "
+ + context.attemptId)
+ }
+ }
+ }
+
+ def readFromFile[T](path: String, context: TaskContext): Iterator[T] = {
+ val inputPath = new Path(path)
+ val fs = inputPath.getFileSystem(new Configuration())
+ val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+ val fileInputStream = fs.open(inputPath, bufferSize)
+ val serializer = SparkEnv.get.serializer.newInstance()
+ val deserializeStream = serializer.deserializeStream(fileInputStream)
+
+ // Register an on-task-completion callback to close the input stream.
+ context.addOnCompleteCallback(() => deserializeStream.close())
+
+ deserializeStream.asIterator.asInstanceOf[Iterator[T]]
+ }
+
+ // Test whether CheckpointRDD generate expected number of splits despite
+ // each split file having multiple blocks. This needs to be run on a
+ // cluster (mesos or standalone) using HDFS.
+ def main(args: Array[String]) {
+ import spark._
+
+ val Array(cluster, hdfsPath) = args
+ val sc = new SparkContext(cluster, "CheckpointRDD Test")
+ val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
+ val path = new Path(hdfsPath, "temp")
+ val fs = path.getFileSystem(new Configuration())
+ sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
+ val cpRDD = new CheckpointRDD[Int](sc, path.toString)
+ assert(cpRDD.splits.length == rdd.splits.length, "Number of splits is not the same")
+ assert(cpRDD.collect.toList == rdd.collect.toList, "Data of splits not the same")
+ fs.delete(path)
+ }
+}
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index ce5f171911..1d528be2aa 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -1,5 +1,6 @@
package spark.rdd
+import java.io.{ObjectOutputStream, IOException}
import java.util.{HashMap => JHashMap}
import scala.collection.JavaConversions
import scala.collection.mutable.ArrayBuffer
@@ -9,7 +10,21 @@ import spark.{Dependency, OneToOneDependency, ShuffleDependency}
private[spark] sealed trait CoGroupSplitDep extends Serializable
-private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
+
+private[spark] case class NarrowCoGroupSplitDep(
+ rdd: RDD[_],
+ splitIndex: Int,
+ var split: Split
+ ) extends CoGroupSplitDep {
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent split at the time of task serialization
+ split = rdd.splits(splitIndex)
+ oos.defaultWriteObject()
+ }
+}
+
private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
private[spark]
@@ -25,30 +40,31 @@ private[spark] class CoGroupAggregator
{ (b1, b2) => b1 ++ b2 })
with Serializable
-class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
- extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
+class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner)
+ extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) with Logging {
val aggr = new CoGroupAggregator
@transient
- override val dependencies = {
+ var deps_ = {
val deps = new ArrayBuffer[Dependency[_]]
for ((rdd, index) <- rdds.zipWithIndex) {
- val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
- if (mapSideCombinedRDD.partitioner == Some(part)) {
- logInfo("Adding one-to-one dependency with " + mapSideCombinedRDD)
- deps += new OneToOneDependency(mapSideCombinedRDD)
+ if (rdd.partitioner == Some(part)) {
+ logInfo("Adding one-to-one dependency with " + rdd)
+ deps += new OneToOneDependency(rdd)
} else {
logInfo("Adding shuffle dependency with " + rdd)
+ val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
deps += new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
}
}
deps.toList
}
+ override def getDependencies = deps_
+
@transient
- val splits_ : Array[Split] = {
- val firstRdd = rdds.head
+ var splits_ : Array[Split] = {
val array = new Array[Split](part.numPartitions)
for (i <- 0 until array.size) {
array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) =>
@@ -56,19 +72,17 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
case s: ShuffleDependency[_, _] =>
new ShuffleCoGroupSplitDep(s.shuffleId): CoGroupSplitDep
case _ =>
- new NarrowCoGroupSplitDep(r, r.splits(i)): CoGroupSplitDep
+ new NarrowCoGroupSplitDep(r, i, r.splits(i)): CoGroupSplitDep
}
}.toList)
}
array
}
- override def splits = splits_
-
+ override def getSplits = splits_
+
override val partitioner = Some(part)
- override def preferredLocations(s: Split) = Nil
-
override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
val split = s.asInstanceOf[CoGroupSplit]
val numRdds = split.deps.size
@@ -84,7 +98,7 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
}
for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
- case NarrowCoGroupSplitDep(rdd, itsSplit) => {
+ case NarrowCoGroupSplitDep(rdd, itsSplitIndex, itsSplit) => {
// Read them from the parent
for ((k, v) <- rdd.iterator(itsSplit, context)) {
getSeq(k.asInstanceOf[K])(depNum) += v
@@ -103,4 +117,10 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
JavaConversions.mapAsScalaMap(map).iterator
}
+
+ override def clearDependencies() {
+ deps_ = null
+ splits_ = null
+ rdds = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
index 1affe0e0ef..167755bbba 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -1,9 +1,22 @@
package spark.rdd
-import spark.{NarrowDependency, RDD, Split, TaskContext}
+import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Split, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
+private[spark] case class CoalescedRDDSplit(
+ index: Int,
+ @transient rdd: RDD[_],
+ parentsIndices: Array[Int]
+ ) extends Split {
+ var parents: Seq[Split] = parentsIndices.map(rdd.splits(_))
-private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent split at the time of task serialization
+ parents = parentsIndices.map(rdd.splits(_))
+ oos.defaultWriteObject()
+ }
+}
/**
* Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
@@ -13,34 +26,44 @@ private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) exten
* This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
* or to avoid having a large number of small tasks when processing a directory with many files.
*/
-class CoalescedRDD[T: ClassManifest](prev: RDD[T], maxPartitions: Int)
- extends RDD[T](prev.context) {
+class CoalescedRDD[T: ClassManifest](
+ var prev: RDD[T],
+ maxPartitions: Int)
+ extends RDD[T](prev.context, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs
- @transient val splits_ : Array[Split] = {
+ @transient var splits_ : Array[Split] = {
val prevSplits = prev.splits
if (prevSplits.length < maxPartitions) {
- prevSplits.zipWithIndex.map{ case (s, idx) => new CoalescedRDDSplit(idx, Array(s)) }
+ prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) }
} else {
(0 until maxPartitions).map { i =>
val rangeStart = (i * prevSplits.length) / maxPartitions
val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions
- new CoalescedRDDSplit(i, prevSplits.slice(rangeStart, rangeEnd))
+ new CoalescedRDDSplit(i, prev, (rangeStart until rangeEnd).toArray)
}.toArray
}
}
- override def splits = splits_
+ override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[T] = {
- split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap {
- parentSplit => prev.iterator(parentSplit, context)
+ split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit =>
+ firstParent[T].iterator(parentSplit, context)
}
}
- val dependencies = List(
+ var deps_ : List[Dependency[_]] = List(
new NarrowDependency(prev) {
def getParents(id: Int): Seq[Int] =
- splits(id).asInstanceOf[CoalescedRDDSplit].parents.map(_.index)
+ splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices
}
)
+
+ override def getDependencies() = deps_
+
+ override def clearDependencies() {
+ deps_ = Nil
+ splits_ = null
+ prev = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala
index d46549b8b6..6dbe235bd9 100644
--- a/core/src/main/scala/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala
@@ -2,11 +2,15 @@ package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
+private[spark] class FilteredRDD[T: ClassManifest](
+ prev: RDD[T],
+ f: T => Boolean)
+ extends RDD[T](prev) {
+
+ override def getSplits = firstParent[T].splits
-private[spark]
-class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
override val partitioner = prev.partitioner // Since filter cannot change a partition's keys
- override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).filter(f)
+
+ override def compute(split: Split, context: TaskContext) =
+ firstParent[T].iterator(split, context).filter(f)
}
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
index 785662b2da..1b604c66e2 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
@@ -1,16 +1,16 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Split, TaskContext}
+
private[spark]
class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: T => TraversableOnce[U])
- extends RDD[U](prev.context) {
+ extends RDD[U](prev) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
+ override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext) =
- prev.iterator(split, context).flatMap(f)
+ firstParent[T].iterator(split, context).flatMap(f)
}
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala
index fac8ffb4cb..051bffed19 100644
--- a/core/src/main/scala/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala
@@ -1,12 +1,12 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Split, TaskContext}
+private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
+ extends RDD[Array[T]](prev) {
+
+ override def getSplits = firstParent[T].splits
-private[spark]
-class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split, context: TaskContext) =
- Array(prev.iterator(split, context).toArray).iterator
-} \ No newline at end of file
+ Array(firstParent[T].iterator(split, context).toArray).iterator
+}
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index ab163f569b..f547f53812 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -22,9 +22,8 @@ import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskCo
* A Spark split class that wraps around a Hadoop InputSplit.
*/
private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
- extends Split
- with Serializable {
-
+ extends Split {
+
val inputSplit = new SerializableWritable[InputSplit](s)
override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
@@ -43,7 +42,7 @@ class HadoopRDD[K, V](
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int)
- extends RDD[(K, V)](sc) {
+ extends RDD[(K, V)](sc, Nil) {
// A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@@ -64,7 +63,7 @@ class HadoopRDD[K, V](
.asInstanceOf[InputFormat[K, V]]
}
- override def splits = splits_
+ override def getSplits = splits_
override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[HadoopSplit]
@@ -110,11 +109,13 @@ class HadoopRDD[K, V](
}
}
- override def preferredLocations(split: Split) = {
+ override def getPreferredLocations(split: Split) = {
// TODO: Filtering out "localhost" in case of file:// URLs
val hadoopSplit = split.asInstanceOf[HadoopSplit]
hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost")
}
- override val dependencies: List[Dependency[_]] = Nil
+ override def checkpoint() {
+ // Do nothing. Hadoop RDD should not be checkpointed.
+ }
}
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
index c764505345..073f7d7d2a 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
@@ -1,6 +1,6 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Split, TaskContext}
private[spark]
@@ -8,11 +8,13 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: Iterator[T] => Iterator[U],
preservesPartitioning: Boolean = false)
- extends RDD[U](prev.context) {
+ extends RDD[U](prev) {
- override val partitioner = if (preservesPartitioning) prev.partitioner else None
+ override val partitioner =
+ if (preservesPartitioning) firstParent[T].partitioner else None
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split, context: TaskContext) = f(prev.iterator(split, context))
+ override def getSplits = firstParent[T].splits
+
+ override def compute(split: Split, context: TaskContext) =
+ f(firstParent[T].iterator(split, context))
} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
index 3d9888bd34..2ddc3d01b6 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
@@ -1,6 +1,7 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Split, TaskContext}
+
/**
* A variant of the MapPartitionsRDD that passes the split index into the
@@ -11,12 +12,13 @@ private[spark]
class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: (Int, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean)
- extends RDD[U](prev.context) {
+ preservesPartitioning: Boolean
+ ) extends RDD[U](prev) {
+
+ override def getSplits = firstParent[T].splits
override val partitioner = if (preservesPartitioning) prev.partitioner else None
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
+
override def compute(split: Split, context: TaskContext) =
- f(split.index, prev.iterator(split, context))
+ f(split.index, firstParent[T].iterator(split, context))
} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala
index 70fa8f4497..c6ceb272cd 100644
--- a/core/src/main/scala/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/MappedRDD.scala
@@ -1,14 +1,15 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Split, TaskContext}
private[spark]
class MappedRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: T => U)
- extends RDD[U](prev.context) {
+ extends RDD[U](prev) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).map(f)
+ override def getSplits = firstParent[T].splits
+
+ override def compute(split: Split, context: TaskContext) =
+ firstParent[T].iterator(split, context).map(f)
} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index 197ed5ea17..bb22db073c 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -20,11 +20,12 @@ class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit
}
class NewHadoopRDD[K, V](
- sc: SparkContext,
+ sc : SparkContext,
inputFormatClass: Class[_ <: InputFormat[K, V]],
- keyClass: Class[K], valueClass: Class[V],
+ keyClass: Class[K],
+ valueClass: Class[V],
@transient conf: Configuration)
- extends RDD[(K, V)](sc)
+ extends RDD[(K, V)](sc, Nil)
with HadoopMapReduceUtil {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
@@ -51,7 +52,7 @@ class NewHadoopRDD[K, V](
result
}
- override def splits = splits_
+ override def getSplits = splits_
override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[NewHadoopSplit]
@@ -86,10 +87,8 @@ class NewHadoopRDD[K, V](
}
}
- override def preferredLocations(split: Split) = {
+ override def getPreferredLocations(split: Split) = {
val theSplit = split.asInstanceOf[NewHadoopSplit]
theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost")
}
-
- override val dependencies: List[Dependency[_]] = Nil
}
diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala
index 336e193217..6631f83510 100644
--- a/core/src/main/scala/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/spark/rdd/PipedRDD.scala
@@ -8,7 +8,7 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
-import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
+import spark.{RDD, SparkEnv, Split, TaskContext}
/**
@@ -16,18 +16,18 @@ import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
* (printing them one per line) and returns the output as a collection of strings.
*/
class PipedRDD[T: ClassManifest](
- parent: RDD[T], command: Seq[String], envVars: Map[String, String])
- extends RDD[String](parent.context) {
+ prev: RDD[T],
+ command: Seq[String],
+ envVars: Map[String, String])
+ extends RDD[String](prev) {
- def this(parent: RDD[T], command: Seq[String]) = this(parent, command, Map())
+ def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map())
// Similar to Runtime.exec(), if we are given a single string, split it into words
// using a standard StringTokenizer (i.e. by spaces)
- def this(parent: RDD[T], command: String) = this(parent, PipedRDD.tokenize(command))
+ def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command))
- override def splits = parent.splits
-
- override val dependencies = List(new OneToOneDependency(parent))
+ override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext): Iterator[String] = {
val pb = new ProcessBuilder(command)
@@ -52,7 +52,7 @@ class PipedRDD[T: ClassManifest](
override def run() {
SparkEnv.set(env)
val out = new PrintWriter(proc.getOutputStream)
- for (elem <- parent.iterator(split, context)) {
+ for (elem <- firstParent[T].iterator(split, context)) {
out.println(elem)
}
out.close()
diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala
index 6e4797aabb..1bc9c96112 100644
--- a/core/src/main/scala/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/spark/rdd/SampledRDD.scala
@@ -1,11 +1,11 @@
package spark.rdd
import java.util.Random
+
import cern.jet.random.Poisson
import cern.jet.random.engine.DRand
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
-
+import spark.{RDD, Split, TaskContext}
private[spark]
class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
@@ -14,23 +14,21 @@ class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Seriali
class SampledRDD[T: ClassManifest](
prev: RDD[T],
- withReplacement: Boolean,
+ withReplacement: Boolean,
frac: Double,
seed: Int)
- extends RDD[T](prev.context) {
+ extends RDD[T](prev) {
@transient
- val splits_ = {
+ var splits_ : Array[Split] = {
val rg = new Random(seed)
- prev.splits.map(x => new SampledRDDSplit(x, rg.nextInt))
+ firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt))
}
- override def splits = splits_.asInstanceOf[Array[Split]]
-
- override val dependencies = List(new OneToOneDependency(prev))
+ override def getSplits = splits_.asInstanceOf[Array[Split]]
- override def preferredLocations(split: Split) =
- prev.preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
+ override def getPreferredLocations(split: Split) =
+ firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
override def compute(splitIn: Split, context: TaskContext) = {
val split = splitIn.asInstanceOf[SampledRDDSplit]
@@ -38,7 +36,7 @@ class SampledRDD[T: ClassManifest](
// For large datasets, the expected number of occurrences of each element in a sample with
// replacement is Poisson(frac). We use that to get a count for each element.
val poisson = new Poisson(frac, new DRand(split.seed))
- prev.iterator(split.prev, context).flatMap { element =>
+ firstParent[T].iterator(split.prev, context).flatMap { element =>
val count = poisson.nextInt()
if (count == 0) {
Iterator.empty // Avoid object allocation when we return 0 items, which is quite often
@@ -48,7 +46,11 @@ class SampledRDD[T: ClassManifest](
}
} else { // Sampling without replacement
val rand = new Random(split.seed)
- prev.iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
+ firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
}
}
+
+ override def clearDependencies() {
+ splits_ = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
index f832633646..1b219473e0 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -1,7 +1,7 @@
package spark.rdd
-import spark.{OneToOneDependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
-
+import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
+import spark.SparkContext._
private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
override val index = idx
@@ -10,28 +10,29 @@ private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
/**
* The resulting RDD from a shuffle (e.g. repartitioning of data).
- * @param parent the parent RDD.
+ * @param prev the parent RDD.
* @param part the partitioner used to partition the RDD
* @tparam K the key class.
* @tparam V the value class.
*/
class ShuffledRDD[K, V](
- @transient parent: RDD[(K, V)],
- part: Partitioner) extends RDD[(K, V)](parent.context) {
+ prev: RDD[(K, V)],
+ part: Partitioner)
+ extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) {
override val partitioner = Some(part)
@transient
- val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
-
- override def splits = splits_
-
- override def preferredLocations(split: Split) = Nil
+ var splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
- val dep = new ShuffleDependency(parent, part)
- override val dependencies = List(dep)
+ override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = {
- SparkEnv.get.shuffleFetcher.fetch[K, V](dep.shuffleId, split.index)
+ val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
+ SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index)
+ }
+
+ override def clearDependencies() {
+ splits_ = null
}
}
diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala
index a08473f7be..24a085df02 100644
--- a/core/src/main/scala/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/spark/rdd/UnionRDD.scala
@@ -1,43 +1,47 @@
package spark.rdd
import scala.collection.mutable.ArrayBuffer
-
import spark.{Dependency, RangeDependency, RDD, SparkContext, Split, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
+private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
+ extends Split {
-private[spark] class UnionSplit[T: ClassManifest](
- idx: Int,
- rdd: RDD[T],
- split: Split)
- extends Split
- with Serializable {
+ var split: Split = rdd.splits(splitIndex)
def iterator(context: TaskContext) = rdd.iterator(split, context)
+
def preferredLocations() = rdd.preferredLocations(split)
+
override val index: Int = idx
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent split at the time of task serialization
+ split = rdd.splits(splitIndex)
+ oos.defaultWriteObject()
+ }
}
class UnionRDD[T: ClassManifest](
sc: SparkContext,
- @transient rdds: Seq[RDD[T]])
- extends RDD[T](sc)
- with Serializable {
+ @transient var rdds: Seq[RDD[T]])
+ extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs
@transient
- val splits_ : Array[Split] = {
+ var splits_ : Array[Split] = {
val array = new Array[Split](rdds.map(_.splits.size).sum)
var pos = 0
for (rdd <- rdds; split <- rdd.splits) {
- array(pos) = new UnionSplit(pos, rdd, split)
+ array(pos) = new UnionSplit(pos, rdd, split.index)
pos += 1
}
array
}
- override def splits = splits_
+ override def getSplits = splits_
- @transient
- override val dependencies = {
+ @transient var deps_ = {
val deps = new ArrayBuffer[Dependency[_]]
var pos = 0
for (rdd <- rdds) {
@@ -47,9 +51,17 @@ class UnionRDD[T: ClassManifest](
deps.toList
}
+ override def getDependencies = deps_
+
override def compute(s: Split, context: TaskContext): Iterator[T] =
s.asInstanceOf[UnionSplit[T]].iterator(context)
- override def preferredLocations(s: Split): Seq[String] =
+ override def getPreferredLocations(s: Split): Seq[String] =
s.asInstanceOf[UnionSplit[T]].preferredLocations()
+
+ override def clearDependencies() {
+ deps_ = null
+ splits_ = null
+ rdds = null
+ }
}
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala
index 92d667ff1e..16e6cc0f1b 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala
@@ -1,53 +1,66 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest](
idx: Int,
- rdd1: RDD[T],
- rdd2: RDD[U],
- split1: Split,
- split2: Split)
- extends Split
- with Serializable {
+ @transient rdd1: RDD[T],
+ @transient rdd2: RDD[U]
+ ) extends Split {
- def iterator(context: TaskContext): Iterator[(T, U)] =
- rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
+ var split1 = rdd1.splits(idx)
+ var split2 = rdd1.splits(idx)
+ override val index: Int = idx
- def preferredLocations(): Seq[String] =
- rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
+ def splits = (split1, split2)
- override val index: Int = idx
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent split at the time of task serialization
+ split1 = rdd1.splits(idx)
+ split2 = rdd2.splits(idx)
+ oos.defaultWriteObject()
+ }
}
class ZippedRDD[T: ClassManifest, U: ClassManifest](
sc: SparkContext,
- @transient rdd1: RDD[T],
- @transient rdd2: RDD[U])
- extends RDD[(T, U)](sc)
+ var rdd1: RDD[T],
+ var rdd2: RDD[U])
+ extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2)))
with Serializable {
+ // TODO: FIX THIS.
+
@transient
- val splits_ : Array[Split] = {
+ var splits_ : Array[Split] = {
if (rdd1.splits.size != rdd2.splits.size) {
throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions")
}
val array = new Array[Split](rdd1.splits.size)
for (i <- 0 until rdd1.splits.size) {
- array(i) = new ZippedSplit(i, rdd1, rdd2, rdd1.splits(i), rdd2.splits(i))
+ array(i) = new ZippedSplit(i, rdd1, rdd2)
}
array
}
- override def splits = splits_
+ override def getSplits = splits_
- @transient
- override val dependencies = List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))
+ override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = {
+ val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
+ rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
+ }
- override def compute(s: Split, context: TaskContext): Iterator[(T, U)] =
- s.asInstanceOf[ZippedSplit[T, U]].iterator(context)
+ override def getPreferredLocations(s: Split): Seq[String] = {
+ val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
+ rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
+ }
- override def preferredLocations(s: Split): Seq[String] =
- s.asInstanceOf[ZippedSplit[T, U]].preferredLocations()
+ override def clearDependencies() {
+ splits_ = null
+ rdd1 = null
+ rdd2 = null
+ }
}
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 29757b1178..59f2099e91 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -14,6 +14,7 @@ import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
import spark.storage.BlockManagerMaster
import spark.storage.BlockManagerId
+import util.{MetadataCleaner, TimeStampedHashMap}
/**
* A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for
@@ -61,9 +62,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
val nextStageId = new AtomicInteger(0)
- val idToStage = new HashMap[Int, Stage]
+ val idToStage = new TimeStampedHashMap[Int, Stage]
- val shuffleToMapStage = new HashMap[Int, Stage]
+ val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
var cacheLocs = new HashMap[Int, Array[List[String]]]
@@ -77,12 +78,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
val running = new HashSet[Stage] // Stages we are running right now
val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures
- val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
+ val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits
val activeJobs = new HashSet[ActiveJob]
val resultStageToJob = new HashMap[Stage, ActiveJob]
+ val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup)
+
// Start a thread to run the DAGScheduler event loop
new Thread("DAGScheduler") {
setDaemon(true)
@@ -594,8 +597,23 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
return Nil
}
+ def cleanup(cleanupTime: Long) {
+ var sizeBefore = idToStage.size
+ idToStage.clearOldValues(cleanupTime)
+ logInfo("idToStage " + sizeBefore + " --> " + idToStage.size)
+
+ sizeBefore = shuffleToMapStage.size
+ shuffleToMapStage.clearOldValues(cleanupTime)
+ logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size)
+
+ sizeBefore = pendingTasks.size
+ pendingTasks.clearOldValues(cleanupTime)
+ logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size)
+ }
+
def stop() {
eventQueue.put(StopDAGScheduler)
+ metadataCleaner.cancel()
taskSched.stop()
}
}
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index 2aad7956b4..8cd4c661eb 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -1,17 +1,74 @@
package spark.scheduler
import spark._
+import java.io._
+import util.{MetadataCleaner, TimeStampedHashMap}
+import java.util.zip.{GZIPInputStream, GZIPOutputStream}
+
+private[spark] object ResultTask {
+
+ // A simple map between the stage id to the serialized byte array of a task.
+ // Served as a cache for task serialization because serialization can be
+ // expensive on the master node if it needs to launch thousands of tasks.
+ val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
+
+ val metadataCleaner = new MetadataCleaner("ResultTask", serializedInfoCache.clearOldValues)
+
+ def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
+ synchronized {
+ val old = serializedInfoCache.get(stageId).orNull
+ if (old != null) {
+ return old
+ } else {
+ val out = new ByteArrayOutputStream
+ val ser = SparkEnv.get.closureSerializer.newInstance
+ val objOut = ser.serializeStream(new GZIPOutputStream(out))
+ objOut.writeObject(rdd)
+ objOut.writeObject(func)
+ objOut.close()
+ val bytes = out.toByteArray
+ serializedInfoCache.put(stageId, bytes)
+ return bytes
+ }
+ }
+ }
+
+ def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = {
+ synchronized {
+ val loader = Thread.currentThread.getContextClassLoader
+ val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
+ val ser = SparkEnv.get.closureSerializer.newInstance
+ val objIn = ser.deserializeStream(in)
+ val rdd = objIn.readObject().asInstanceOf[RDD[_]]
+ val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _]
+ return (rdd, func)
+ }
+ }
+
+ def clearCache() {
+ synchronized {
+ serializedInfoCache.clear()
+ }
+ }
+}
+
private[spark] class ResultTask[T, U](
stageId: Int,
- rdd: RDD[T],
- func: (TaskContext, Iterator[T]) => U,
- val partition: Int,
+ var rdd: RDD[T],
+ var func: (TaskContext, Iterator[T]) => U,
+ var partition: Int,
@transient locs: Seq[String],
val outputId: Int)
- extends Task[U](stageId) {
+ extends Task[U](stageId) with Externalizable {
- val split = rdd.splits(partition)
+ def this() = this(0, null, null, 0, null, 0)
+
+ var split = if (rdd == null) {
+ null
+ } else {
+ rdd.splits(partition)
+ }
override def run(attemptId: Long): U = {
val context = new TaskContext(stageId, partition, attemptId)
@@ -25,4 +82,31 @@ private[spark] class ResultTask[T, U](
override def preferredLocations: Seq[String] = locs
override def toString = "ResultTask(" + stageId + ", " + partition + ")"
+
+ override def writeExternal(out: ObjectOutput) {
+ RDDCheckpointData.synchronized {
+ split = rdd.splits(partition)
+ out.writeInt(stageId)
+ val bytes = ResultTask.serializeInfo(
+ stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
+ out.writeInt(bytes.length)
+ out.write(bytes)
+ out.writeInt(partition)
+ out.writeInt(outputId)
+ out.writeObject(split)
+ }
+ }
+
+ override def readExternal(in: ObjectInput) {
+ val stageId = in.readInt()
+ val numBytes = in.readInt()
+ val bytes = new Array[Byte](numBytes)
+ in.readFully(bytes)
+ val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes)
+ rdd = rdd_.asInstanceOf[RDD[T]]
+ func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
+ partition = in.readInt()
+ val outputId = in.readInt()
+ split = in.readObject().asInstanceOf[Split]
+ }
}
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index bd1911fce2..19f5328eee 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -14,17 +14,20 @@ import com.ning.compress.lzf.LZFOutputStream
import spark._
import spark.storage._
+import util.{TimeStampedHashMap, MetadataCleaner}
private[spark] object ShuffleMapTask {
// A simple map between the stage id to the serialized byte array of a task.
// Served as a cache for task serialization because serialization can be
// expensive on the master node if it needs to launch thousands of tasks.
- val serializedInfoCache = new JHashMap[Int, Array[Byte]]
+ val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
+
+ val metadataCleaner = new MetadataCleaner("ShuffleMapTask", serializedInfoCache.clearOldValues)
def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
synchronized {
- val old = serializedInfoCache.get(stageId)
+ val old = serializedInfoCache.get(stageId).orNull
if (old != null) {
return old
} else {
@@ -87,13 +90,16 @@ private[spark] class ShuffleMapTask(
}
override def writeExternal(out: ObjectOutput) {
- out.writeInt(stageId)
- val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
- out.writeInt(bytes.length)
- out.write(bytes)
- out.writeInt(partition)
- out.writeLong(generation)
- out.writeObject(split)
+ RDDCheckpointData.synchronized {
+ split = rdd.splits(partition)
+ out.writeInt(stageId)
+ val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
+ out.writeInt(bytes.length)
+ out.write(bytes)
+ out.writeInt(partition)
+ out.writeLong(generation)
+ out.writeObject(split)
+ }
}
override def readExternal(in: ObjectInput) {
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
index 2593c0e3a0..dff550036d 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -81,7 +81,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
ser.serialize(Accumulators.values))
logInfo("Finished task " + idInJob)
- listener.taskEnded(task, Success, resultToReturn, accumUpdates)
+
+ // If the threadpool has not already been shutdown, notify DAGScheduler
+ if (!Thread.currentThread().isInterrupted)
+ listener.taskEnded(task, Success, resultToReturn, accumUpdates)
} catch {
case t: Throwable => {
logError("Exception in task " + idInJob, t)
@@ -91,7 +94,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
submitTask(task, idInJob)
} else {
// TODO: Do something nicer here to return all the way to the user
- listener.taskEnded(task, new ExceptionFailure(t), null, null)
+ if (!Thread.currentThread().isInterrupted)
+ listener.taskEnded(task, new ExceptionFailure(t), null, null)
}
}
}
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
index 00e32f753c..ae88ff0bb1 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -17,7 +17,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
private var currentMemory = 0L
-
// Object used to ensure that only one thread is putting blocks and if necessary, dropping
// blocks from the memory store.
private val putLock = new Object()
diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala
index 19e67acd0c..139e21d09e 100644
--- a/core/src/main/scala/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/spark/util/MetadataCleaner.scala
@@ -4,9 +4,10 @@ import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
import java.util.{TimerTask, Timer}
import spark.Logging
+
class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
- val delaySeconds = (System.getProperty("spark.cleanup.delay", "-100").toDouble * 60).toInt
+ val delaySeconds = MetadataCleaner.getDelaySeconds
val periodSeconds = math.max(10, delaySeconds / 10)
val timer = new Timer(name + " cleanup timer", true)
@@ -22,6 +23,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
}
}
}
+
if (periodSeconds > 0) {
logInfo(
"Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and "
@@ -33,3 +35,10 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
timer.cancel()
}
}
+
+
+object MetadataCleaner {
+ def getDelaySeconds = (System.getProperty("spark.cleaner.delay", "-100").toDouble * 60).toInt
+ def setDelaySeconds(delay: Long) { System.setProperty("spark.cleaner.delay", delay.toString) }
+}
+
diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
new file mode 100644
index 0000000000..e3f00ea8c7
--- /dev/null
+++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
@@ -0,0 +1,62 @@
+package spark.util
+
+import scala.annotation.tailrec
+
+import java.io.OutputStream
+import java.util.concurrent.TimeUnit._
+
+class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends OutputStream {
+ val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS)
+ val CHUNK_SIZE = 8192
+ var lastSyncTime = System.nanoTime
+ var bytesWrittenSinceSync: Long = 0
+
+ override def write(b: Int) {
+ waitToWrite(1)
+ out.write(b)
+ }
+
+ override def write(bytes: Array[Byte]) {
+ write(bytes, 0, bytes.length)
+ }
+
+ @tailrec
+ override final def write(bytes: Array[Byte], offset: Int, length: Int) {
+ val writeSize = math.min(length - offset, CHUNK_SIZE)
+ if (writeSize > 0) {
+ waitToWrite(writeSize)
+ out.write(bytes, offset, writeSize)
+ write(bytes, offset + writeSize, length)
+ }
+ }
+
+ override def flush() {
+ out.flush()
+ }
+
+ override def close() {
+ out.close()
+ }
+
+ @tailrec
+ private def waitToWrite(numBytes: Int) {
+ val now = System.nanoTime
+ val elapsedSecs = SECONDS.convert(math.max(now - lastSyncTime, 1), NANOSECONDS)
+ val rate = bytesWrittenSinceSync.toDouble / elapsedSecs
+ if (rate < bytesPerSec) {
+ // It's okay to write; just update some variables and return
+ bytesWrittenSinceSync += numBytes
+ if (now > lastSyncTime + SYNC_INTERVAL) {
+ // Sync interval has passed; let's resync
+ lastSyncTime = now
+ bytesWrittenSinceSync = numBytes
+ }
+ } else {
+ // Calculate how much time we should sleep to bring ourselves to the desired rate.
+ // Based on throttler in Kafka (https://github.com/kafka-dev/kafka/blob/master/core/src/main/scala/kafka/utils/Throttler.scala)
+ val sleepTime = MILLISECONDS.convert((bytesWrittenSinceSync / bytesPerSec - elapsedSecs), SECONDS)
+ if (sleepTime > 0) Thread.sleep(sleepTime)
+ waitToWrite(numBytes)
+ }
+ }
+}
diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
index 070ee19ac0..bb7c5c01c8 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
@@ -1,16 +1,16 @@
package spark.util
import java.util.concurrent.ConcurrentHashMap
-import scala.collection.JavaConversions._
-import scala.collection.mutable.{HashMap, Map}
+import scala.collection.JavaConversions
+import scala.collection.mutable.Map
/**
* This is a custom implementation of scala.collection.mutable.Map which stores the insertion
* time stamp along with each key-value pair. Key-value pairs that are older than a particular
- * threshold time can them be removed using the cleanup method. This is intended to be a drop-in
+ * threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in
* replacement of scala.collection.mutable.HashMap.
*/
-class TimeStampedHashMap[A, B] extends Map[A, B]() {
+class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging {
val internalMap = new ConcurrentHashMap[A, (B, Long)]()
def get(key: A): Option[B] = {
@@ -20,7 +20,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
def iterator: Iterator[(A, B)] = {
val jIterator = internalMap.entrySet().iterator()
- jIterator.map(kv => (kv.getKey, kv.getValue._1))
+ JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1))
}
override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = {
@@ -31,8 +31,10 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
override def - (key: A): Map[A, B] = {
- internalMap.remove(key)
- this
+ val newMap = new TimeStampedHashMap[A, B]
+ newMap.internalMap.putAll(this.internalMap)
+ newMap.internalMap.remove(key)
+ newMap
}
override def += (kv: (A, B)): this.type = {
@@ -56,7 +58,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
override def filter(p: ((A, B)) => Boolean): Map[A, B] = {
- internalMap.map(kv => (kv._1, kv._2._1)).filter(p)
+ JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
}
override def empty: Map[A, B] = new TimeStampedHashMap[A, B]()
@@ -72,11 +74,15 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
}
- def cleanup(threshTime: Long) {
+ /**
+ * Removes old key-value pairs that have timestamp earlier than `threshTime`
+ */
+ def clearOldValues(threshTime: Long) {
val iterator = internalMap.entrySet().iterator()
while(iterator.hasNext) {
val entry = iterator.next()
if (entry.getValue._2 < threshTime) {
+ logDebug("Removing key " + entry.getKey)
iterator.remove()
}
}
diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
new file mode 100644
index 0000000000..5f1cc93752
--- /dev/null
+++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
@@ -0,0 +1,69 @@
+package spark.util
+
+import scala.collection.mutable.Set
+import scala.collection.JavaConversions
+import java.util.concurrent.ConcurrentHashMap
+
+
+class TimeStampedHashSet[A] extends Set[A] {
+ val internalMap = new ConcurrentHashMap[A, Long]()
+
+ def contains(key: A): Boolean = {
+ internalMap.contains(key)
+ }
+
+ def iterator: Iterator[A] = {
+ val jIterator = internalMap.entrySet().iterator()
+ JavaConversions.asScalaIterator(jIterator).map(_.getKey)
+ }
+
+ override def + (elem: A): Set[A] = {
+ val newSet = new TimeStampedHashSet[A]
+ newSet ++= this
+ newSet += elem
+ newSet
+ }
+
+ override def - (elem: A): Set[A] = {
+ val newSet = new TimeStampedHashSet[A]
+ newSet ++= this
+ newSet -= elem
+ newSet
+ }
+
+ override def += (key: A): this.type = {
+ internalMap.put(key, currentTime)
+ this
+ }
+
+ override def -= (key: A): this.type = {
+ internalMap.remove(key)
+ this
+ }
+
+ override def empty: Set[A] = new TimeStampedHashSet[A]()
+
+ override def size(): Int = internalMap.size()
+
+ override def foreach[U](f: (A) => U): Unit = {
+ val iterator = internalMap.entrySet().iterator()
+ while(iterator.hasNext) {
+ f(iterator.next.getKey)
+ }
+ }
+
+ /**
+ * Removes old values that have timestamp earlier than `threshTime`
+ */
+ def clearOldValues(threshTime: Long) {
+ val iterator = internalMap.entrySet().iterator()
+ while(iterator.hasNext) {
+ val entry = iterator.next()
+ if (entry.getValue < threshTime) {
+ iterator.remove()
+ }
+ }
+ }
+
+ private def currentTime: Long = System.currentTimeMillis()
+}
diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
index 4c99e450bc..6ec89c0184 100644
--- a/core/src/test/resources/log4j.properties
+++ b/core/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the file core/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=core/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala
new file mode 100644
index 0000000000..51573254ca
--- /dev/null
+++ b/core/src/test/scala/spark/CheckpointSuite.scala
@@ -0,0 +1,357 @@
+package spark
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import java.io.File
+import spark.rdd._
+import spark.SparkContext._
+import storage.StorageLevel
+
+class CheckpointSuite extends FunSuite with BeforeAndAfter with Logging {
+ initLogging()
+
+ var sc: SparkContext = _
+ var checkpointDir: File = _
+ val partitioner = new HashPartitioner(2)
+
+ before {
+ checkpointDir = File.createTempFile("temp", "")
+ checkpointDir.delete()
+
+ sc = new SparkContext("local", "test")
+ sc.setCheckpointDir(checkpointDir.toString)
+ }
+
+ after {
+ if (sc != null) {
+ sc.stop()
+ sc = null
+ }
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+
+ if (checkpointDir != null) {
+ checkpointDir.delete()
+ }
+ }
+
+ test("RDDs with one-to-one dependencies") {
+ testCheckpointing(_.map(x => x.toString))
+ testCheckpointing(_.flatMap(x => 1 to x))
+ testCheckpointing(_.filter(_ % 2 == 0))
+ testCheckpointing(_.sample(false, 0.5, 0))
+ testCheckpointing(_.glom())
+ testCheckpointing(_.mapPartitions(_.map(_.toString)))
+ testCheckpointing(r => new MapPartitionsWithSplitRDD(r,
+ (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
+ testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
+ testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
+ testCheckpointing(_.pipe(Seq("cat")))
+ }
+
+ test("ParallelCollection") {
+ val parCollection = sc.makeRDD(1 to 4, 2)
+ val numSplits = parCollection.splits.size
+ parCollection.checkpoint()
+ assert(parCollection.dependencies === Nil)
+ val result = parCollection.collect()
+ assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result)
+ assert(parCollection.dependencies != Nil)
+ assert(parCollection.splits.length === numSplits)
+ assert(parCollection.splits.toList === parCollection.checkpointData.get.getSplits.toList)
+ assert(parCollection.collect() === result)
+ }
+
+ test("BlockRDD") {
+ val blockId = "id"
+ val blockManager = SparkEnv.get.blockManager
+ blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
+ val blockRDD = new BlockRDD[String](sc, Array(blockId))
+ val numSplits = blockRDD.splits.size
+ blockRDD.checkpoint()
+ val result = blockRDD.collect()
+ assert(sc.checkpointFile[String](blockRDD.getCheckpointFile.get).collect() === result)
+ assert(blockRDD.dependencies != Nil)
+ assert(blockRDD.splits.length === numSplits)
+ assert(blockRDD.splits.toList === blockRDD.checkpointData.get.getSplits.toList)
+ assert(blockRDD.collect() === result)
+ }
+
+ test("ShuffledRDD") {
+ testCheckpointing(rdd => {
+ // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
+ new ShuffledRDD(rdd.map(x => (x % 2, 1)), partitioner)
+ })
+ }
+
+ test("UnionRDD") {
+ def otherRDD = sc.makeRDD(1 to 10, 1)
+
+ // Test whether the size of UnionRDDSplits reduce in size after parent RDD is checkpointed.
+ // Current implementation of UnionRDD has transient reference to parent RDDs,
+ // so only the splits will reduce in serialized size, not the RDD.
+ testCheckpointing(_.union(otherRDD), false, true)
+ testParentCheckpointing(_.union(otherRDD), false, true)
+ }
+
+ test("CartesianRDD") {
+ def otherRDD = sc.makeRDD(1 to 10, 1)
+ testCheckpointing(new CartesianRDD(sc, _, otherRDD))
+
+ // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
+ // Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
+ // so only the RDD will reduce in serialized size, not the splits.
+ testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
+
+ // Test that the CartesianRDD updates parent splits (CartesianRDD.s1/s2) after
+ // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
+ // Note that this test is very specific to the current implementation of CartesianRDD.
+ val ones = sc.makeRDD(1 to 100, 10).map(x => x)
+ ones.checkpoint // checkpoint that MappedRDD
+ val cartesian = new CartesianRDD(sc, ones, ones)
+ val splitBeforeCheckpoint =
+ serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
+ cartesian.count() // do the checkpointing
+ val splitAfterCheckpoint =
+ serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
+ assert(
+ (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
+ (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
+ "CartesianRDD.parents not updated after parent RDD checkpointed"
+ )
+ }
+
+ test("CoalescedRDD") {
+ testCheckpointing(new CoalescedRDD(_, 2))
+
+ // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
+ // Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
+ // so only the RDD will reduce in serialized size, not the splits.
+ testParentCheckpointing(new CoalescedRDD(_, 2), true, false)
+
+ // Test that the CoalescedRDDSplit updates parent splits (CoalescedRDDSplit.parents) after
+ // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
+ // Note that this test is very specific to the current implementation of CoalescedRDDSplits
+ val ones = sc.makeRDD(1 to 100, 10).map(x => x)
+ ones.checkpoint // checkpoint that MappedRDD
+ val coalesced = new CoalescedRDD(ones, 2)
+ val splitBeforeCheckpoint =
+ serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
+ coalesced.count() // do the checkpointing
+ val splitAfterCheckpoint =
+ serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
+ assert(
+ splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
+ "CoalescedRDDSplit.parents not updated after parent RDD checkpointed"
+ )
+ }
+
+ test("CoGroupedRDD") {
+ val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD()
+ testCheckpointing(rdd => {
+ CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner)
+ }, false, true)
+
+ val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD()
+ testParentCheckpointing(rdd => {
+ CheckpointSuite.cogroup(
+ longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner)
+ }, false, true)
+ }
+
+ test("ZippedRDD") {
+ testCheckpointing(
+ rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+
+ // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
+ // Current implementation of ZippedRDDSplit has transient references to parent RDDs,
+ // so only the RDD will reduce in serialized size, not the splits.
+ testParentCheckpointing(
+ rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+
+ }
+
+ /**
+ * 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.
+ * It can also test whether the size of serialized RDD splits has reduced after checkpointing or
+ * not, but this is not done by default as usually the splits do not refer to any RDD and
+ * therefore never store the lineage.
+ */
+ def testCheckpointing[U: ClassManifest](
+ op: (RDD[Int]) => RDD[U],
+ testRDDSize: Boolean = true,
+ testRDDSplitSize: Boolean = false
+ ) {
+ // Generate the final RDD using given RDD operation
+ val baseRDD = generateLongLineageRDD
+ val operatedRDD = op(baseRDD)
+ val parentRDD = operatedRDD.dependencies.headOption.orNull
+ val rddType = operatedRDD.getClass.getSimpleName
+ val numSplits = operatedRDD.splits.length
+
+ // Find serialized sizes before and after the checkpoint
+ val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+ operatedRDD.checkpoint()
+ val result = operatedRDD.collect()
+ val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+
+ // Test whether the checkpoint file has been created
+ assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result)
+
+ // Test whether dependencies have been changed from its earlier parent RDD
+ assert(operatedRDD.dependencies.head.rdd != parentRDD)
+
+ // Test whether the splits have been changed to the new Hadoop splits
+ assert(operatedRDD.splits.toList === operatedRDD.checkpointData.get.getSplits.toList)
+
+ // Test whether the number of splits is same as before
+ assert(operatedRDD.splits.length === numSplits)
+
+ // Test whether the data in the checkpointed RDD is same as original
+ assert(operatedRDD.collect() === result)
+
+ // Test whether serialized size of the RDD has reduced. If the RDD
+ // does not have any dependency to another RDD (e.g., ParallelCollection,
+ // ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing.
+ if (testRDDSize) {
+ logInfo("Size of " + rddType +
+ "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
+ assert(
+ rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+ "Size of " + rddType + " did not reduce after checkpointing " +
+ "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+ )
+ }
+
+ // Test whether serialized size of the splits has reduced. If the splits
+ // do not have any non-transient reference to another RDD or another RDD's splits, it
+ // does not refer to a lineage and therefore may not reduce in size after checkpointing.
+ // However, if the original splits before checkpointing do refer to a parent RDD, the splits
+ // must be forgotten after checkpointing (to remove all reference to parent RDDs) and
+ // replaced with the HadoopSplits of the checkpointed RDD.
+ if (testRDDSplitSize) {
+ logInfo("Size of " + rddType + " splits "
+ + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
+ assert(
+ splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
+ "Size of " + rddType + " splits did not reduce after checkpointing " +
+ "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
+ )
+ }
+ }
+
+ /**
+ * Test whether checkpointing of the parent of the generated RDD also
+ * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
+ * RDDs splits. So even if the parent RDD is checkpointed and its splits changed,
+ * this RDD will remember the splits and therefore potentially the whole lineage.
+ */
+ def testParentCheckpointing[U: ClassManifest](
+ op: (RDD[Int]) => RDD[U],
+ testRDDSize: Boolean,
+ testRDDSplitSize: Boolean
+ ) {
+ // Generate the final RDD using given RDD operation
+ val baseRDD = generateLongLineageRDD
+ val operatedRDD = op(baseRDD)
+ val parentRDD = operatedRDD.dependencies.head.rdd
+ val rddType = operatedRDD.getClass.getSimpleName
+ val parentRDDType = parentRDD.getClass.getSimpleName
+
+ // Find serialized sizes before and after the checkpoint
+ val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+ parentRDD.checkpoint() // checkpoint the parent RDD, not the generated one
+ val result = operatedRDD.collect()
+ val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+
+ // Test whether the data in the checkpointed RDD is same as original
+ assert(operatedRDD.collect() === result)
+
+ // Test whether serialized size of the RDD has reduced because of its parent being
+ // checkpointed. If this RDD or its parent RDD do not have any dependency
+ // to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may
+ // not reduce in size after checkpointing.
+ if (testRDDSize) {
+ assert(
+ rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+ "Size of " + rddType + " did not reduce after parent checkpointing parent " + parentRDDType +
+ "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+ )
+ }
+
+ // Test whether serialized size of the splits has reduced because of its parent being
+ // checkpointed. If the splits do not have any non-transient reference to another RDD
+ // or another RDD's splits, it does not refer to a lineage and therefore may not reduce
+ // in size after checkpointing. However, if the splits do refer to the *splits* of a parent
+ // RDD, then these splits must update reference to the parent RDD splits as the parent RDD's
+ // splits must have changed after checkpointing.
+ if (testRDDSplitSize) {
+ assert(
+ splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
+ "Size of " + rddType + " splits did not reduce after checkpointing parent " + parentRDDType +
+ "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
+ )
+ }
+
+ }
+
+ /**
+ * Generate an RDD with a long lineage of one-to-one dependencies.
+ */
+ def generateLongLineageRDD(): RDD[Int] = {
+ var rdd = sc.makeRDD(1 to 100, 4)
+ for (i <- 1 to 50) {
+ rdd = rdd.map(x => x + 1)
+ }
+ rdd
+ }
+
+ /**
+ * Generate an RDD with a long lineage specifically for CoGroupedRDD.
+ * A CoGroupedRDD can have a long lineage only one of its parents have a long lineage
+ * and narrow dependency with this RDD. This method generate such an RDD by a sequence
+ * of cogroups and mapValues which creates a long lineage of narrow dependencies.
+ */
+ def generateLongLineageRDDForCoGroupedRDD() = {
+ val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _)
+
+ def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+
+ var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones)
+ for(i <- 1 to 10) {
+ cogrouped = cogrouped.mapValues(add).cogroup(ones)
+ }
+ cogrouped.mapValues(add)
+ }
+
+ /**
+ * Get serialized sizes of the RDD and its splits
+ */
+ def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
+ (Utils.serialize(rdd).size, Utils.serialize(rdd.splits).size)
+ }
+
+ /**
+ * Serialize and deserialize an object. This is useful to verify the objects
+ * contents after deserialization (e.g., the contents of an RDD split after
+ * it is sent to a slave along with a task)
+ */
+ def serializeDeserialize[T](obj: T): T = {
+ val bytes = Utils.serialize(obj)
+ Utils.deserialize[T](bytes)
+ }
+}
+
+
+object CheckpointSuite {
+ // This is a custom cogroup function that does not use mapValues like
+ // the PairRDDFunctions.cogroup()
+ def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = {
+ //println("First = " + first + ", second = " + second)
+ new CoGroupedRDD[K](
+ Seq(first.asInstanceOf[RDD[(_, _)]], second.asInstanceOf[RDD[(_, _)]]),
+ part
+ ).asInstanceOf[RDD[(K, Seq[Seq[V]])]]
+ }
+
+}
diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala
index 7c0334d957..dfa2de80e6 100644
--- a/core/src/test/scala/spark/ClosureCleanerSuite.scala
+++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala
@@ -47,6 +47,8 @@ object TestObject {
val nums = sc.parallelize(Array(1, 2, 3, 4))
val answer = nums.map(_ + x).reduce(_ + _)
sc.stop()
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
return answer
}
}
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index c61913fc82..01351de4ae 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -586,7 +586,7 @@ public class JavaAPISuite implements Serializable {
public void accumulators() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
- final Accumulator<Integer> intAccum = sc.accumulator(10);
+ final Accumulator<Integer> intAccum = sc.intAccumulator(10);
rdd.foreach(new VoidFunction<Integer>() {
public void call(Integer x) {
intAccum.add(x);
@@ -594,7 +594,7 @@ public class JavaAPISuite implements Serializable {
});
Assert.assertEquals((Integer) 25, intAccum.value());
- final Accumulator<Double> doubleAccum = sc.accumulator(10.0);
+ final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
rdd.foreach(new VoidFunction<Integer>() {
public void call(Integer x) {
doubleAccum.add((double) x);
@@ -641,4 +641,31 @@ public class JavaAPISuite implements Serializable {
Assert.assertEquals(new Tuple2<String, Integer>("1", 1), s.get(0));
Assert.assertEquals(new Tuple2<String, Integer>("2", 2), s.get(1));
}
+
+ @Test
+ public void checkpointAndComputation() {
+ File tempDir = Files.createTempDir();
+ JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+ sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+ Assert.assertEquals(false, rdd.isCheckpointed());
+ rdd.checkpoint();
+ rdd.count(); // Forces the DAG to cause a checkpoint
+ Assert.assertEquals(true, rdd.isCheckpointed());
+ Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
+ }
+
+ @Test
+ public void checkpointAndRestore() {
+ File tempDir = Files.createTempDir();
+ JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+ sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+ Assert.assertEquals(false, rdd.isCheckpointed());
+ rdd.checkpoint();
+ rdd.count(); // Forces the DAG to cause a checkpoint
+ Assert.assertEquals(true, rdd.isCheckpointed());
+
+ Assert.assertTrue(rdd.getCheckpointFile().isPresent());
+ JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
+ Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
+ }
}
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index d74e9786c3..db217f8482 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -76,10 +76,23 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5)))
}
- test("checkpointing") {
+ test("basic checkpointing") {
+ import java.io.File
+ val checkpointDir = File.createTempFile("temp", "")
+ checkpointDir.delete()
+
sc = new SparkContext("local", "test")
- val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).flatMap(x => 1 to x).checkpoint()
- assert(rdd.collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4))
+ sc.setCheckpointDir(checkpointDir.toString)
+ val parCollection = sc.makeRDD(1 to 4)
+ val flatMappedRDD = parCollection.flatMap(x => 1 to x)
+ flatMappedRDD.checkpoint()
+ assert(flatMappedRDD.dependencies.head.rdd == parCollection)
+ val result = flatMappedRDD.collect()
+ Thread.sleep(1000)
+ assert(flatMappedRDD.dependencies.head.rdd != parCollection)
+ assert(flatMappedRDD.collect() === result)
+
+ checkpointDir.deleteOnExit()
}
test("basic caching") {
@@ -94,9 +107,9 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
sc = new SparkContext("local", "test")
val onlySplit = new Split { override def index: Int = 0 }
var shouldFail = true
- val rdd = new RDD[Int](sc) {
- override def splits: Array[Split] = Array(onlySplit)
- override val dependencies = List[Dependency[_]]()
+ val rdd = new RDD[Int](sc, Nil) {
+ override def getSplits: Array[Split] = Array(onlySplit)
+ override val getDependencies = List[Dependency[_]]()
override def compute(split: Split, context: TaskContext): Iterator[Int] = {
if (shouldFail) {
throw new Exception("injected failure")
@@ -123,8 +136,8 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10)))
// Check that the narrow dependency is also specified correctly
- assert(coalesced1.dependencies.head.getParents(0).toList === List(0, 1, 2, 3, 4))
- assert(coalesced1.dependencies.head.getParents(1).toList === List(5, 6, 7, 8, 9))
+ assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === List(0, 1, 2, 3, 4))
+ assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === List(5, 6, 7, 8, 9))
val coalesced2 = new CoalescedRDD(data, 3)
assert(coalesced2.collect().toList === (1 to 10).toList)
diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
index f937877340..ba6f8b588f 100644
--- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
@@ -23,9 +23,8 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter {
test("Calls executeOnCompleteCallbacks after failure") {
var completed = false
sc = new SparkContext("local", "test")
- val rdd = new RDD[String](sc) {
- override val splits = Array[Split](StubSplit(0))
- override val dependencies = List()
+ val rdd = new RDD[String](sc, List()) {
+ override def getSplits = Array[Split](StubSplit(0))
override def compute(split: Split, context: TaskContext) = {
context.addOnCompleteCallback(() => completed = true)
sys.error("failed")
diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
new file mode 100644
index 0000000000..794063fb6d
--- /dev/null
+++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
@@ -0,0 +1,23 @@
+package spark.util
+
+import org.scalatest.FunSuite
+import java.io.ByteArrayOutputStream
+import java.util.concurrent.TimeUnit._
+
+class RateLimitedOutputStreamSuite extends FunSuite {
+
+ private def benchmark[U](f: => U): Long = {
+ val start = System.nanoTime
+ f
+ System.nanoTime - start
+ }
+
+ test("write") {
+ val underlying = new ByteArrayOutputStream
+ val data = "X" * 41000
+ val stream = new RateLimitedOutputStream(underlying, 10000)
+ val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) }
+ assert(SECONDS.convert(elapsedNs, NANOSECONDS) == 4)
+ assert(underlying.toString("UTF-8") == data)
+ }
+}
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 9804d449fc..94baa634aa 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -48,14 +48,16 @@
<li><a href="scala-programming-guide.html">Scala</a></li>
<li><a href="java-programming-guide.html">Java</a></li>
<li><a href="python-programming-guide.html">Python</a></li>
+ <li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
</ul>
</li>
-
+
<li class="dropdown">
- <a href="#" class="dropdown-toggle" data-toggle="dropdown">API<b class="caret"></b></a>
+ <a href="#" class="dropdown-toggle" data-toggle="dropdown">API (Scaladoc)<b class="caret"></b></a>
<ul class="dropdown-menu">
- <li><a href="api/core/index.html">Scala/Java (Scaladoc)</a></li>
- <li><a href="api/pyspark/index.html">Python (Epydoc)</a></li>
+ <li><a href="api/core/index.html">Spark Scala/Java (Scaladoc)</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>
</ul>
</li>
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index c9ce589c1b..e400dec619 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -2,7 +2,7 @@ require 'fileutils'
include FileUtils
if ENV['SKIP_SCALADOC'] != '1'
- projects = ["core", "examples", "repl", "bagel"]
+ projects = ["core", "examples", "repl", "bagel", "streaming"]
puts "Moving to project root and building scaladoc."
curr_dir = pwd
@@ -11,7 +11,7 @@ if ENV['SKIP_SCALADOC'] != '1'
puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..."
puts `sbt/sbt doc`
- puts "moving back into docs dir."
+ puts "Moving back into docs dir."
cd("docs")
# Copy over the scaladoc from each project into the docs directory.
diff --git a/docs/api.md b/docs/api.md
index b9c93ac5e8..e86d07770a 100644
--- a/docs/api.md
+++ b/docs/api.md
@@ -5,7 +5,8 @@ title: Spark API documentation (Scaladoc)
Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory.
-- [Core](api/core/index.html)
-- [Examples](api/examples/index.html)
+- [Spark](api/core/index.html)
+- [Spark Examples](api/examples/index.html)
+- [Spark Streaming](api/streaming/index.html)
- [Bagel](api/bagel/index.html)
- [PySpark](api/pyspark/index.html)
diff --git a/docs/configuration.md b/docs/configuration.md
index d8317ea97c..87cb4a6797 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -219,6 +219,17 @@ Apart from these, the following properties are also available, and may be useful
Port for the master to listen on.
</td>
</tr>
+<tr>
+ <td>spark.cleaner.delay</td>
+ <td>(disable)</td>
+ <td>
+ Duration (minutes) 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
+ 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.
+ </td>
+</tr>
+
</table>
# Configuring Logging
diff --git a/docs/index.md b/docs/index.md
index 848b585333..c6ef507cb0 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -58,13 +58,15 @@ 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!
* [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
* [Python Programming Guide](python-programming-guide.html): using Spark from Python
**API Docs:**
-* [Java/Scala (Scaladoc)](api/core/index.html)
-* [Python (Epydoc)](api/pyspark/index.html)
+* [Spark Java/Scala (Scaladoc)](api/core/index.html)
+* [Spark Python (Epydoc)](api/pyspark/index.html)
+* [Spark Streaming Java/Scala (Scaladoc)](api/streaming/index.html)
**Deployment guides:**
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
new file mode 100644
index 0000000000..b6da7af654
--- /dev/null
+++ b/docs/streaming-programming-guide.md
@@ -0,0 +1,313 @@
+---
+layout: global
+title: Spark Streaming Programming Guide
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+# 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:
+(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.
+(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.
+
+This guide shows some how to start programming with DStreams.
+
+# Initializing Spark Streaming
+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 %}
+new StreamingContext(master, jobName, batchDuration)
+{% 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.
+
+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 %}
+
+
+
+# Attaching Input Sources - InputDStreams
+The StreamingContext is used to creating InputDStreams from input sources:
+
+{% highlight scala %}
+// 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
+{% 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.
+
+
+
+# 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.
+
+## Transformations
+
+DStreams support many of the transformations available on normal Spark RDD's:
+
+<table class="table">
+<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <td> <b>mapPartitions</b>(<i>func</i>) </td>
+ <td> Similar to map, but runs separately on each partition (block) of the DStream, so <i>func</i> must be of type
+ Iterator[T] => Iterator[U] when running on an DStream of type T. </td>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <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 />
+<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.
+</td>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <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>
+</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>
+ <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>
+</tr>
+</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.
+
+<table class="table">
+<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
+<tr>
+ <td> <b>window</b>(<i>windowDuration</i>, </i>slideTime</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>
+</tr>
+<tr>
+ <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideTime</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>
+</tr>
+<tr>
+ <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>
+</tr>
+<tr>
+ <td> <b>groupByKeyAndWindow</b>(windowDuration, slideDuration, [<i>numTasks</i>])
+ </td>
+ <td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs over 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>.
+</td>
+</tr>
+<tr>
+ <td> <b>reduceByKeyAndWindow</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 over batches 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>.
+</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>
+
+</table>
+
+A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#spark.streaming.PairDStreamFunctions).
+
+## Output Operations
+When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined:
+
+<table class="table">
+<tr><th style="width:25%">Operator</th><th>Meaning</th></tr>
+<tr>
+ <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>
+</tr>
+
+<tr>
+ <td> <b>print</b>() </td>
+ <td> Prints first ten elements of every batch of data in a DStream on the driver. </td>
+</tr>
+
+<tr>
+ <td> <b>saveAsObjectFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a <code>SequenceFile</code> of serialized objects. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>.
+ </td>
+</tr>
+
+<tr>
+ <td> <b>saveAsTextFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a text files. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+
+<tr>
+ <td> <b>saveAsHadoopFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+
+</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
+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 %}
+ssc.start()
+{% endhighlight %}
+
+Conversely, the computation can be stopped by using
+{% highlight scala %}
+ssc.stop()
+{% endhighlight %}
+
+# Example - NetworkWordCount.scala
+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.
+
+{% highlight scala %}
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+...
+
+// 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 lines = ssc.networkTextStream(args(1), args(2).toInt)
+
+// Split the lines into words, count them, and print some of the counts on the master
+val words = lines.flatMap(_.split(" "))
+val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+wordCounts.print()
+
+// Start the computation
+ssc.start()
+{% endhighlight %}
+
+To run this example on your local machine, you need to first run a Netcat server by using
+
+{% highlight bash %}
+$ nc -lk 9999
+{% endhighlight %}
+
+Then, in a different terminal, you can start NetworkWordCount by using
+
+{% highlight bash %}
+$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+{% endhighlight %}
+
+This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
+
+<table>
+<td>
+{% highlight bash %}
+# TERMINAL 1
+# RUNNING NETCAT
+
+$ nc -lk 9999
+hello world
+
+
+
+
+
+...
+{% endhighlight %}
+</td>
+<td>
+{% highlight bash %}
+# TERMINAL 2: RUNNING NetworkWordCount
+...
+2012-12-31 18:47:10,446 INFO SparkContext: Job finished: run at ThreadPoolExecutor.java:886, took 0.038817 s
+-------------------------------------------
+Time: 1357008430000 ms
+-------------------------------------------
+(hello,1)
+(world,1)
+
+2012-12-31 18:47:10,447 INFO JobManager: Total delay: 0.44700 s for job 8 (execution: 0.44000 s)
+...
+{% endhighlight %}
+</td>
+</table>
+
+
+
+# Performance Tuning
+Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
+<ol>
+<li>Reducing the processing time of each batch of data by efficiently using cluster resources.</li>
+<li>Setting the right batch size such that the data processing can keep up with the data ingestion.</li>
+</ol>
+
+## Reducing the Processing Time of each Batch
+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
+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.
+
+### 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.
+* 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
+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.
+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
+For a Spark Streaming application running on a cluster to be stable, the processing of the data streams must keep up with the rate of ingestion of the data streams. Depending on the type of computation, the batch size used may have significant impact on the rate of ingestion that can be sustained by the Spark Streaming application on a fixed cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not every 500 milliseconds.
+
+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
+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.
+
+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
+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.
+
+* <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.
+
+# Master Fault-tolerance (Alpha)
+TODO
+
+* Checkpointing of DStream graph
+
+* Recovery from master faults
+
+* Current state and future directions \ No newline at end of file
diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
new file mode 100644
index 0000000000..461929fba2
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
@@ -0,0 +1,43 @@
+package spark.streaming.examples
+
+import spark.util.IntParam
+import spark.storage.StorageLevel
+import spark.streaming._
+
+/**
+ * Produces a count of events received from Flume.
+ *
+ * This should be used in conjunction with an AvroSink in Flume. It will start
+ * an Avro server on at the request host:port address and listen for requests.
+ * Your Flume AvroSink should be pointed to this address.
+ *
+ * Usage: FlumeEventCount <master> <host> <port>
+ *
+ * <master> is a Spark master URL
+ * <host> is the host the Flume receiver will be started on - a receiver
+ * creates a server and listens for flume events.
+ * <port> is the port the Flume receiver will listen on.
+ */
+object FlumeEventCount {
+ def main(args: Array[String]) {
+ if (args.length != 3) {
+ System.err.println(
+ "Usage: FlumeEventCount <master> <host> <port>")
+ System.exit(1)
+ }
+
+ val Array(master, host, IntParam(port)) = args
+
+ val batchInterval = Milliseconds(2000)
+ // Create the context and set the batch size
+ val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval)
+
+ // Create a flume stream
+ val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
+
+ // Print out the count of events received from this server in each batch
+ stream.count().map(cnt => "Received " + cnt + " flume events." ).print()
+
+ ssc.start()
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
new file mode 100644
index 0000000000..8530f5c175
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
@@ -0,0 +1,36 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+
+/**
+ * Counts words in new text files created in the given directory
+ * Usage: HdfsWordCount <master> <directory>
+ * <master> is the Spark master URL.
+ * <directory> is the directory that Spark Streaming will use to find and read new text files.
+ *
+ * To run this on your local machine on directory `localdir`, run this example
+ * `$ ./run spark.streaming.examples.HdfsWordCount local[2] localdir`
+ * Then create a text file in `localdir` and the words in the file will get counted.
+ */
+object HdfsWordCount {
+ def main(args: Array[String]) {
+ if (args.length < 2) {
+ System.err.println("Usage: HdfsWordCount <master> <directory>")
+ System.exit(1)
+ }
+
+ // Create the context
+ val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2))
+
+ // Create the FileInputDStream on the directory and use the
+ // stream to count words in new files created
+ val lines = ssc.textFileStream(args(1))
+ val words = lines.flatMap(_.split(" "))
+ val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+ wordCounts.print()
+ ssc.start()
+ }
+}
+
diff --git a/examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java
new file mode 100644
index 0000000000..cddce16e39
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java
@@ -0,0 +1,50 @@
+package spark.streaming.examples;
+
+import spark.api.java.function.Function;
+import spark.streaming.*;
+import spark.streaming.api.java.*;
+import spark.streaming.dstream.SparkFlumeEvent;
+
+/**
+ * Produces a count of events received from Flume.
+ *
+ * This should be used in conjunction with an AvroSink in Flume. It will start
+ * an Avro server on at the request host:port address and listen for requests.
+ * Your Flume AvroSink should be pointed to this address.
+ *
+ * Usage: JavaFlumeEventCount <master> <host> <port>
+ *
+ * <master> is a Spark master URL
+ * <host> is the host the Flume receiver will be started on - a receiver
+ * creates a server and listens for flume events.
+ * <port> is the port the Flume receiver will listen on.
+ */
+public class JavaFlumeEventCount {
+ public static void main(String[] args) {
+ if (args.length != 3) {
+ System.err.println("Usage: JavaFlumeEventCount <master> <host> <port>");
+ System.exit(1);
+ }
+
+ String master = args[0];
+ String host = args[1];
+ int port = Integer.parseInt(args[2]);
+
+ Duration batchInterval = new Duration(2000);
+
+ JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval);
+
+ JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
+
+ flumeStream.count();
+
+ flumeStream.count().map(new Function<Long, String>() {
+ @Override
+ public String call(Long in) {
+ return "Received " + in + " flume events.";
+ }
+ }).print();
+
+ sc.start();
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java
new file mode 100644
index 0000000000..4299febfd6
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java
@@ -0,0 +1,62 @@
+package spark.streaming.examples;
+
+import com.google.common.collect.Lists;
+import scala.Tuple2;
+import spark.api.java.function.FlatMapFunction;
+import spark.api.java.function.Function2;
+import spark.api.java.function.PairFunction;
+import spark.streaming.Duration;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ * Usage: NetworkWordCount <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 TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
+ */
+public class JavaNetworkWordCount {
+ public static void main(String[] args) {
+ if (args.length < 2) {
+ System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
+ "In local mode, <master> should be 'local[n]' with n > 1");
+ System.exit(1);
+ }
+
+ // Create the context with a 1 second batch size
+ JavaStreamingContext ssc = new JavaStreamingContext(
+ args[0], "NetworkWordCount", new Duration(1000));
+
+ // Create a NetworkInputDStream on target ip:port and count the
+ // 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> words = lines.flatMap(new FlatMapFunction<String, String>() {
+ @Override
+ public Iterable<String> call(String x) {
+ return Lists.newArrayList(x.split(" "));
+ }
+ });
+ 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;
+ }
+ });
+
+ wordCounts.print();
+ ssc.start();
+
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java
new file mode 100644
index 0000000000..43c3cd4dfa
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java
@@ -0,0 +1,62 @@
+package spark.streaming.examples;
+
+import com.google.common.collect.Lists;
+import scala.Tuple2;
+import spark.api.java.JavaRDD;
+import spark.api.java.function.Function2;
+import spark.api.java.function.PairFunction;
+import spark.streaming.Duration;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+public class JavaQueueStream {
+ public static void main(String[] args) throws InterruptedException {
+ if (args.length < 1) {
+ System.err.println("Usage: JavaQueueStream <master>");
+ System.exit(1);
+ }
+
+ // Create the context
+ JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000));
+
+ // Create the queue through which RDDs can be pushed to
+ // a QueueInputDStream
+ Queue<JavaRDD<Integer>> rddQueue = new LinkedList<JavaRDD<Integer>>();
+
+ // Create and push some RDDs into the queue
+ List<Integer> list = Lists.newArrayList();
+ for (int i = 0; i < 1000; i++) {
+ list.add(i);
+ }
+
+ for (int i = 0; i < 30; i++) {
+ rddQueue.add(ssc.sc().parallelize(list));
+ }
+
+
+ // Create the QueueInputDStream and use it do some processing
+ JavaDStream<Integer> inputStream = ssc.queueStream(rddQueue);
+ JavaPairDStream<Integer, Integer> mappedStream = inputStream.map(
+ new PairFunction<Integer, Integer, Integer>() {
+ @Override
+ public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+ return new Tuple2<Integer, Integer>(i % 10, 1);
+ }
+ });
+ JavaPairDStream<Integer, Integer> reducedStream = mappedStream.reduceByKey(
+ new Function2<Integer, Integer, Integer>() {
+ @Override
+ public Integer call(Integer i1, Integer i2) throws Exception {
+ return i1 + i2;
+ }
+ });
+
+ reducedStream.print();
+ ssc.start();
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
new file mode 100644
index 0000000000..fe55db6e2c
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
@@ -0,0 +1,69 @@
+package spark.streaming.examples
+
+import java.util.Properties
+import kafka.message.Message
+import kafka.producer.SyncProducerConfig
+import kafka.producer._
+import spark.SparkContext
+import spark.streaming._
+import spark.streaming.StreamingContext._
+import spark.storage.StorageLevel
+import spark.streaming.util.RawTextHelper._
+
+object KafkaWordCount {
+ def main(args: Array[String]) {
+
+ if (args.length < 6) {
+ System.err.println("Usage: KafkaWordCount <master> <hostname> <port> <group> <topics> <numThreads>")
+ System.exit(1)
+ }
+
+ val Array(master, hostname, port, group, topics, numThreads) = args
+
+ val sc = new SparkContext(master, "KafkaWordCount")
+ val ssc = new StreamingContext(sc, Seconds(2))
+ ssc.checkpoint("checkpoint")
+
+ val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
+ val lines = ssc.kafkaStream[String](hostname, port.toInt, group, topicpMap)
+ val words = lines.flatMap(_.split(" "))
+ val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
+ wordCounts.print()
+
+ ssc.start()
+ }
+}
+
+// Produces some random words between 1 and 100.
+object KafkaWordCountProducer {
+
+ def main(args: Array[String]) {
+ if (args.length < 3) {
+ System.err.println("Usage: KafkaWordCountProducer <hostname> <port> <topic> <messagesPerSec> <wordsPerMessage>")
+ System.exit(1)
+ }
+
+ val Array(hostname, port, topic, messagesPerSec, wordsPerMessage) = args
+
+ // Zookeper connection properties
+ val props = new Properties()
+ props.put("zk.connect", hostname + ":" + port)
+ props.put("serializer.class", "kafka.serializer.StringEncoder")
+
+ val config = new ProducerConfig(props)
+ val producer = new Producer[String, String](config)
+
+ // Send some messages
+ while(true) {
+ val messages = (1 to messagesPerSec.toInt).map { messageNum =>
+ (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString).mkString(" ")
+ }.toArray
+ println(messages.mkString(","))
+ val data = new ProducerData[String, String](topic, messages)
+ producer.send(data)
+ Thread.sleep(100)
+ }
+ }
+
+}
+
diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
new file mode 100644
index 0000000000..32f7d57bea
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
@@ -0,0 +1,36 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ * Usage: NetworkWordCount <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 TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ */
+object NetworkWordCount {
+ def main(args: Array[String]) {
+ if (args.length < 2) {
+ System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
+ "In local mode, <master> should be 'local[n]' with n > 1")
+ System.exit(1)
+ }
+
+ // Create the context with a 1 second batch size
+ val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1))
+
+ // Create a NetworkInputDStream on target ip:port and count the
+ // words in input stream of \n delimited test (eg. generated by 'nc')
+ val lines = ssc.networkTextStream(args(1), args(2).toInt)
+ val words = lines.flatMap(_.split(" "))
+ val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+ wordCounts.print()
+ ssc.start()
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
new file mode 100644
index 0000000000..2a265d021d
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
@@ -0,0 +1,39 @@
+package spark.streaming.examples
+
+import spark.RDD
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+import scala.collection.mutable.SynchronizedQueue
+
+object QueueStream {
+
+ def main(args: Array[String]) {
+ if (args.length < 1) {
+ System.err.println("Usage: QueueStream <master>")
+ System.exit(1)
+ }
+
+ // Create the context
+ val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1))
+
+ // Create the queue through which RDDs can be pushed to
+ // a QueueInputDStream
+ val rddQueue = new SynchronizedQueue[RDD[Int]]()
+
+ // Create the QueueInputDStream and use it do some processing
+ val inputStream = ssc.queueStream(rddQueue)
+ val mappedStream = inputStream.map(x => (x % 10, 1))
+ val reducedStream = mappedStream.reduceByKey(_ + _)
+ reducedStream.print()
+ ssc.start()
+
+ // Create and push some RDDs into
+ for (i <- 1 to 30) {
+ rddQueue += ssc.sc.makeRDD(1 to 1000, 10)
+ Thread.sleep(1000)
+ }
+ ssc.stop()
+ System.exit(0)
+ }
+} \ No newline at end of file
diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
new file mode 100644
index 0000000000..2eec777c54
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
@@ -0,0 +1,46 @@
+package spark.streaming.examples
+
+import spark.util.IntParam
+import spark.storage.StorageLevel
+
+import spark.streaming._
+import spark.streaming.util.RawTextHelper
+
+/**
+ * Receives text from multiple rawNetworkStreams and counts how many '\n' delimited
+ * lines have the word 'the' in them. This is useful for benchmarking purposes. This
+ * will only work with spark.streaming.util.RawTextSender running on all worker nodes
+ * and with Spark using Kryo serialization (set Java property "spark.serializer" to
+ * "spark.KryoSerializer").
+ * Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>
+ * <master> is the Spark master URL
+ * <numStream> is the number rawNetworkStreams, which should be same as number
+ * of work nodes in the cluster
+ * <host> is "localhost".
+ * <port> is the port on which RawTextSender is running in the worker nodes.
+ * <batchMillise> is the Spark Streaming batch duration in milliseconds.
+ */
+
+object RawNetworkGrep {
+ def main(args: Array[String]) {
+ if (args.length != 5) {
+ System.err.println("Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>")
+ System.exit(1)
+ }
+
+ val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args
+
+ // Create the context
+ val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis))
+
+ // Warm up the JVMs on master and slave for JIT compilation to kick in
+ RawTextHelper.warmUp(ssc.sc)
+
+ val rawStreams = (1 to numStreams).map(_ =>
+ ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
+ val union = ssc.union(rawStreams)
+ union.filter(_.contains("the")).count().foreach(r =>
+ println("Grep count: " + r.collect().mkString))
+ ssc.start()
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
new file mode 100644
index 0000000000..4c6e08bc74
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -0,0 +1,85 @@
+package spark.streaming.examples.clickstream
+
+import java.net.{InetAddress,ServerSocket,Socket,SocketException}
+import java.io.{InputStreamReader, BufferedReader, PrintWriter}
+import util.Random
+
+/** Represents a page view on a website with associated dimension data.*/
+class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) {
+ override def toString() : String = {
+ "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID)
+ }
+}
+object PageView {
+ def fromString(in : String) : PageView = {
+ val parts = in.split("\t")
+ new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt)
+ }
+}
+
+/** Generates streaming events to simulate page views on a website.
+ *
+ * This should be used in tandem with PageViewStream.scala. Example:
+ * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+ * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * */
+object PageViewGenerator {
+ val pages = Map("http://foo.com/" -> .7,
+ "http://foo.com/news" -> 0.2,
+ "http://foo.com/contact" -> .1)
+ val httpStatus = Map(200 -> .95,
+ 404 -> .05)
+ val userZipCode = Map(94709 -> .5,
+ 94117 -> .5)
+ val userID = Map((1 to 100).map(_ -> .01):_*)
+
+
+ def pickFromDistribution[T](inputMap : Map[T, Double]) : T = {
+ val rand = new Random().nextDouble()
+ var total = 0.0
+ for ((item, prob) <- inputMap) {
+ total = total + prob
+ if (total > rand) {
+ return item
+ }
+ }
+ return inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0
+ }
+
+ def getNextClickEvent() : String = {
+ val id = pickFromDistribution(userID)
+ val page = pickFromDistribution(pages)
+ val status = pickFromDistribution(httpStatus)
+ val zipCode = pickFromDistribution(userZipCode)
+ new PageView(page, status, zipCode, id).toString()
+ }
+
+ def main(args : Array[String]) {
+ if (args.length != 2) {
+ System.err.println("Usage: PageViewGenerator <port> <viewsPerSecond>")
+ System.exit(1)
+ }
+ val port = args(0).toInt
+ val viewsPerSecond = args(1).toFloat
+ val sleepDelayMs = (1000.0 / viewsPerSecond).toInt
+ val listener = new ServerSocket(port)
+ println("Listening on port: " + port)
+
+ while (true) {
+ val socket = listener.accept()
+ new Thread() {
+ override def run = {
+ println("Got client connected from: " + socket.getInetAddress)
+ val out = new PrintWriter(socket.getOutputStream(), true)
+
+ while (true) {
+ Thread.sleep(sleepDelayMs)
+ out.write(getNextClickEvent())
+ out.flush()
+ }
+ socket.close()
+ }
+ }.start()
+ }
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
new file mode 100644
index 0000000000..a191321d91
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -0,0 +1,84 @@
+package spark.streaming.examples.clickstream
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+import spark.SparkContext._
+
+/** Analyses a streaming dataset of web page views. This class demonstrates several types of
+ * operators available in Spark streaming.
+ *
+ * This should be used in tandem with PageViewStream.scala. Example:
+ * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+ * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * */
+object PageViewStream {
+ def main(args: Array[String]) {
+ if (args.length != 3) {
+ System.err.println("Usage: PageViewStream <metric> <host> <port>")
+ System.err.println("<metric> must be one of pageCounts, slidingPageCounts," +
+ " errorRatePerZipCode, activeUserCount, popularUsersSeen")
+ System.exit(1)
+ }
+ val metric = args(0)
+ val host = args(1)
+ val port = args(2).toInt
+
+ // Create the context
+ val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1))
+
+ // Create a NetworkInputDStream on target host:port and convert each line to a PageView
+ val pageViews = ssc.networkTextStream(host, port)
+ .flatMap(_.split("\n"))
+ .map(PageView.fromString(_))
+
+ // Return a count of views per URL seen in each batch
+ val pageCounts = pageViews.map(view => ((view.url, 1))).countByKey()
+
+ // Return a sliding window of page views per URL in the last ten seconds
+ val slidingPageCounts = pageViews.map(view => ((view.url, 1)))
+ .window(Seconds(10), Seconds(2))
+ .countByKey()
+
+
+ // Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds
+ val statusesPerZipCode = pageViews.window(Seconds(30), Seconds(2))
+ .map(view => ((view.zipCode, view.status)))
+ .groupByKey()
+ val errorRatePerZipCode = statusesPerZipCode.map{
+ case(zip, statuses) =>
+ val normalCount = statuses.filter(_ == 200).size
+ val errorCount = statuses.size - normalCount
+ val errorRatio = errorCount.toFloat / statuses.size
+ if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)}
+ else {"%s: %s".format(zip, errorRatio)}
+ }
+
+ // Return the number unique users in last 15 seconds
+ val activeUserCount = pageViews.window(Seconds(15), Seconds(2))
+ .map(view => (view.userID, 1))
+ .groupByKey()
+ .count()
+ .map("Unique active users: " + _)
+
+ // An external dataset we want to join to this stream
+ val userList = ssc.sc.parallelize(
+ Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq)
+
+ metric match {
+ case "pageCounts" => pageCounts.print()
+ case "slidingPageCounts" => slidingPageCounts.print()
+ case "errorRatePerZipCode" => errorRatePerZipCode.print()
+ case "activeUserCount" => activeUserCount.print()
+ case "popularUsersSeen" =>
+ // Look for users in our existing dataset and print it out if we have a match
+ pageViews.map(view => (view.userID, 1))
+ .foreach((rdd, time) => rdd.join(userList)
+ .map(_._2._2)
+ .take(10)
+ .foreach(u => println("Saw user %s at time %s".format(u, time))))
+ case _ => println("Invalid metric entered: " + metric)
+ }
+
+ ssc.start()
+ }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala
new file mode 100644
index 0000000000..377bc0c98e
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala
@@ -0,0 +1,60 @@
+package spark.streaming.examples.twitter
+
+import spark.streaming.StreamingContext._
+import spark.streaming.{Seconds, StreamingContext}
+import spark.SparkContext._
+import spark.storage.StorageLevel
+
+/**
+ * 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
+ * command line arguments.
+ */
+object TwitterBasic {
+ def main(args: Array[String]) {
+ if (args.length < 3) {
+ System.err.println("Usage: TwitterBasic <master> <twitter_username> <twitter_password>" +
+ " [filter1] [filter2] ... [filter n]")
+ System.exit(1)
+ }
+
+ val Array(master, username, password) = args.slice(0, 3)
+ val filters = args.slice(3, args.length)
+
+ val ssc = new StreamingContext(master, "TwitterBasic", Seconds(2))
+ val stream = new TwitterInputDStream(ssc, username, password, filters,
+ StorageLevel.MEMORY_ONLY_SER)
+ ssc.registerInputStream(stream)
+
+ val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
+
+ val topCounts60 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(60))
+ .map{case (topic, count) => (count, topic)}
+ .transform(_.sortByKey(false))
+
+ val topCounts10 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(10))
+ .map{case (topic, count) => (count, topic)}
+ .transform(_.sortByKey(false))
+
+
+ // Print popular hashtags
+ topCounts60.foreach(rdd => {
+ if (rdd.count() != 0) {
+ val topList = rdd.take(5)
+ println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
+ topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
+ }
+ })
+
+ topCounts10.foreach(rdd => {
+ if (rdd.count() != 0) {
+ val topList = rdd.take(5)
+ println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
+ topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
+ }
+ })
+
+ ssc.start()
+ }
+
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala
new file mode 100644
index 0000000000..99ed4cdc1c
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala
@@ -0,0 +1,71 @@
+package spark.streaming.examples.twitter
+
+import spark._
+import spark.streaming._
+import dstream.{NetworkReceiver, NetworkInputDStream}
+import storage.StorageLevel
+import twitter4j._
+import twitter4j.auth.BasicAuthorization
+import collection.JavaConversions._
+
+/* A stream of Twitter statuses, potentially filtered by one or more keywords.
+*
+* @constructor create a new Twitter stream using the supplied username and password to authenticate.
+* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is
+* such that this may return a sampled subset of all tweets during each interval.
+*/
+class TwitterInputDStream(
+ @transient ssc_ : StreamingContext,
+ username: String,
+ password: String,
+ filters: Seq[String],
+ storageLevel: StorageLevel
+ ) extends NetworkInputDStream[Status](ssc_) {
+
+ override def createReceiver(): NetworkReceiver[Status] = {
+ new TwitterReceiver(username, password, filters, storageLevel)
+ }
+}
+
+class TwitterReceiver(
+ username: String,
+ password: String,
+ filters: Seq[String],
+ storageLevel: StorageLevel
+ ) extends NetworkReceiver[Status] {
+
+ var twitterStream: TwitterStream = _
+ lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+ protected override def onStart() {
+ blockGenerator.start()
+ twitterStream = new TwitterStreamFactory()
+ .getInstance(new BasicAuthorization(username, password))
+ twitterStream.addListener(new StatusListener {
+ def onStatus(status: Status) = {
+ blockGenerator += status
+ }
+ // Unimplemented
+ def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {}
+ def onTrackLimitationNotice(i: Int) {}
+ def onScrubGeo(l: Long, l1: Long) {}
+ def onStallWarning(stallWarning: StallWarning) {}
+ def onException(e: Exception) {}
+ })
+
+ val query: FilterQuery = new FilterQuery
+ if (filters.size > 0) {
+ query.track(filters.toArray)
+ twitterStream.filter(query)
+ } else {
+ twitterStream.sample()
+ }
+ logInfo("Twitter receiver started")
+ }
+
+ protected override def onStop() {
+ blockGenerator.stop()
+ twitterStream.shutdown()
+ logInfo("Twitter receiver stopped")
+ }
+}
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 219674028e..3dbb993f9c 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -17,16 +17,18 @@ object SparkBuild extends Build {
//val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1"
//val HADOOP_MAJOR_VERSION = "2"
- lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel)
+ lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming)
lazy val core = Project("core", file("core"), settings = coreSettings)
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core)
- lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core)
+ lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core)
+ lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core)
+
// A configuration to set an alternative publishLocalConfiguration
lazy val MavenCompile = config("m2r") extend(Compile)
lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
@@ -90,7 +92,8 @@ object SparkBuild extends Build {
"org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011",
"org.scalatest" %% "scalatest" % "1.8" % "test",
"org.scalacheck" %% "scalacheck" % "1.9" % "test",
- "com.novocode" % "junit-interface" % "0.8" % "test"
+ "com.novocode" % "junit-interface" % "0.8" % "test",
+ "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile"
),
parallelExecution := false,
/* Workaround for issue #206 (fixed after SBT 0.11.0) */
@@ -114,7 +117,8 @@ object SparkBuild extends Build {
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
"JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
"Spray Repository" at "http://repo.spray.cc/",
- "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
+ "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/",
+ "Twitter4J Repository" at "http://twitter4j.org/maven2/"
),
libraryDependencies ++= Seq(
@@ -132,6 +136,8 @@ object SparkBuild extends Build {
"com.typesafe.akka" % "akka-slf4j" % "2.0.3",
"it.unimi.dsi" % "fastutil" % "6.4.4",
"colt" % "colt" % "1.2.0",
+ "org.twitter4j" % "twitter4j-core" % "3.0.2",
+ "org.twitter4j" % "twitter4j-stream" % "3.0.2",
"cc.spray" % "spray-can" % "1.0-M2.1",
"cc.spray" % "spray-server" % "1.0-M2.1",
"cc.spray" %% "spray-json" % "1.1.1",
@@ -155,6 +161,12 @@ object SparkBuild extends Build {
def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
+ def streamingSettings = sharedSettings ++ Seq(
+ name := "spark-streaming",
+ libraryDependencies ++= Seq(
+ "com.github.sgroschupf" % "zkclient" % "0.1")
+ ) ++ assemblySettings ++ extraAssemblySettings
+
def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
mergeStrategy in assembly := {
case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
index 4c99e450bc..cfb1a390e6 100644
--- a/repl/src/test/resources/log4j.properties
+++ b/repl/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the repl/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=repl/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/run b/run
index eb93db66db..060856007f 100755
--- a/run
+++ b/run
@@ -63,6 +63,7 @@ CORE_DIR="$FWDIR/core"
REPL_DIR="$FWDIR/repl"
EXAMPLES_DIR="$FWDIR/examples"
BAGEL_DIR="$FWDIR/bagel"
+STREAMING_DIR="$FWDIR/streaming"
PYSPARK_DIR="$FWDIR/python"
# Exit if the user hasn't compiled Spark
@@ -82,6 +83,7 @@ fi
CLASSPATH+=":$CORE_DIR/src/main/resources"
CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
+CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
if [ -e "$FWDIR/lib_managed" ]; then
CLASSPATH+=":$FWDIR/lib_managed/jars/*"
CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
diff --git a/streaming/lib/kafka-0.7.2.jar b/streaming/lib/kafka-0.7.2.jar
new file mode 100644
index 0000000000..65f79925a4
--- /dev/null
+++ b/streaming/lib/kafka-0.7.2.jar
Binary files differ
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
new file mode 100644
index 0000000000..2f3adb39c2
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -0,0 +1,118 @@
+package spark.streaming
+
+import spark.{Logging, Utils}
+
+import org.apache.hadoop.fs.{FileUtil, Path}
+import org.apache.hadoop.conf.Configuration
+
+import java.io._
+
+
+private[streaming]
+class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
+ extends Logging with Serializable {
+ val master = ssc.sc.master
+ val framework = ssc.sc.jobName
+ val sparkHome = ssc.sc.sparkHome
+ val jars = ssc.sc.jars
+ val graph = ssc.graph
+ val checkpointDir = ssc.checkpointDir
+ val checkpointDuration: Duration = ssc.checkpointDuration
+
+ def validate() {
+ assert(master != null, "Checkpoint.master is null")
+ assert(framework != null, "Checkpoint.framework is null")
+ assert(graph != null, "Checkpoint.graph is null")
+ assert(checkpointTime != null, "Checkpoint.checkpointTime is null")
+ logInfo("Checkpoint for time " + checkpointTime + " validated")
+ }
+}
+
+/**
+ * Convenience class to speed up the writing of graph checkpoint to file
+ */
+private[streaming]
+class CheckpointWriter(checkpointDir: String) extends Logging {
+ val file = new Path(checkpointDir, "graph")
+ val conf = new Configuration()
+ var fs = file.getFileSystem(conf)
+ val maxAttempts = 3
+
+ def write(checkpoint: Checkpoint) {
+ // TODO: maybe do this in a different thread from the main stream execution thread
+ var attempts = 0
+ while (attempts < maxAttempts) {
+ attempts += 1
+ try {
+ logDebug("Saving checkpoint for time " + checkpoint.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)
+ 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 + "'")
+ }
+}
+
+
+private[streaming]
+object CheckpointReader extends Logging {
+
+ def read(path: String): Checkpoint = {
+ val fs = new Path(path).getFileSystem(new Configuration())
+ val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+
+ attempts.foreach(file => {
+ if (fs.exists(file)) {
+ logInfo("Attempting to load checkpoint from file '" + file + "'")
+ try {
+ val fis = fs.open(file)
+ // ObjectInputStream uses the last defined user-defined class loader in the stack
+ // to find classes, which maybe the wrong class loader. Hence, a inherited version
+ // 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
+ // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
+ val ois = new ObjectInputStreamWithLoader(fis, Thread.currentThread().getContextClassLoader)
+ val cp = ois.readObject.asInstanceOf[Checkpoint]
+ ois.close()
+ fs.close()
+ cp.validate()
+ logInfo("Checkpoint successfully loaded from file '" + file + "'")
+ logInfo("Checkpoint was generated at time " + cp.checkpointTime)
+ return cp
+ } catch {
+ case e: Exception =>
+ logError("Error loading checkpoint from file '" + file + "'", e)
+ }
+ } else {
+ logWarning("Could not read checkpoint from file '" + file + "' as it does not exist")
+ }
+
+ })
+ throw new Exception("Could not read checkpoint from path '" + path + "'")
+ }
+}
+
+private[streaming]
+class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) extends ObjectInputStream(inputStream_) {
+ override def resolveClass(desc: ObjectStreamClass): Class[_] = {
+ try {
+ return loader.loadClass(desc.getName())
+ } catch {
+ case e: Exception =>
+ }
+ return super.resolveClass(desc)
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
new file mode 100644
index 0000000000..b11ef443dc
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -0,0 +1,657 @@
+package spark.streaming
+
+import spark.streaming.dstream._
+import StreamingContext._
+//import Time._
+
+import spark.{RDD, Logging}
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashMap
+
+import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.conf.Configuration
+
+/**
+ * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * for more details on RDDs). DStreams can either be created from live data (such as, data from
+ * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
+ * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
+ * DStream periodically generates a RDD, either from live data or by transforming the RDD generated
+ * by a parent DStream.
+ *
+ * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
+ * `window`. In addition, [[spark.streaming.PairDStreamFunctions]] contains operations available
+ * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations
+ * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through
+ * implicit conversions when `spark.streaming.StreamingContext._` is imported.
+ *
+ * DStreams internally is characterized by a few basic properties:
+ * - A list of other DStreams that the DStream depends on
+ * - A time interval at which the DStream generates an RDD
+ * - A function that is used to generate an RDD after each time interval
+ */
+
+abstract class DStream[T: ClassManifest] (
+ @transient protected[streaming] var ssc: StreamingContext
+ ) extends Serializable with Logging {
+
+ initLogging()
+
+ // =======================================================================
+ // Methods that should be implemented by subclasses of DStream
+ // =======================================================================
+
+ /** Time interval after which the DStream generates a RDD */
+ def slideDuration: Duration
+
+ /** List of parent DStreams on which this DStream depends on */
+ def dependencies: List[DStream[_]]
+
+ /** Method that generates a RDD for the given time */
+ def compute (validTime: Time): Option[RDD[T]]
+
+ // =======================================================================
+ // Methods and fields available on all DStreams
+ // =======================================================================
+
+ // RDDs generated, marked as protected[streaming] so that testsuites can access it
+ @transient
+ protected[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] ()
+
+ // Time zero for the DStream
+ protected[streaming] var zeroTime: Time = null
+
+ // Duration for which the DStream will remember each RDD created
+ protected[streaming] var rememberDuration: Duration = null
+
+ // Storage level of the RDDs in the stream
+ protected[streaming] var storageLevel: StorageLevel = StorageLevel.NONE
+
+ // Checkpoint details
+ protected[streaming] val mustCheckpoint = false
+ protected[streaming] var checkpointDuration: Duration = null
+ protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]())
+
+ // Reference to whole DStream graph
+ protected[streaming] var graph: DStreamGraph = null
+
+ protected[streaming] def isInitialized = (zeroTime != null)
+
+ // Duration for which the DStream requires its parent DStream to remember each RDD created
+ protected[streaming] def parentRememberDuration = rememberDuration
+
+ /** Returns the StreamingContext associated with this DStream */
+ def context() = ssc
+
+ /** Persists the RDDs of this DStream with the given storage level */
+ def persist(level: StorageLevel): DStream[T] = {
+ if (this.isInitialized) {
+ throw new UnsupportedOperationException(
+ "Cannot change storage level of an DStream after streaming context has started")
+ }
+ this.storageLevel = level
+ this
+ }
+
+ /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def persist(): DStream[T] = persist(StorageLevel.MEMORY_ONLY_SER)
+
+ /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def cache(): DStream[T] = persist()
+
+ /**
+ * Enable periodic checkpointing of RDDs of this DStream
+ * @param interval Time interval after which generated RDD will be checkpointed
+ */
+ def checkpoint(interval: Duration): DStream[T] = {
+ if (isInitialized) {
+ throw new UnsupportedOperationException(
+ "Cannot change checkpoint interval of an DStream after streaming context has started")
+ }
+ persist()
+ checkpointDuration = interval
+ this
+ }
+
+ /**
+ * Initialize the DStream by setting the "zero" time, based on which
+ * the validity of future times is calculated. This method also recursively initializes
+ * its parent DStreams.
+ */
+ protected[streaming] def initialize(time: Time) {
+ if (zeroTime != null && zeroTime != time) {
+ throw new Exception("ZeroTime is already initialized to " + zeroTime
+ + ", cannot initialize it again to " + time)
+ }
+ zeroTime = time
+
+ // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger
+ if (mustCheckpoint && checkpointDuration == null) {
+ checkpointDuration = slideDuration.max(Seconds(10))
+ logInfo("Checkpoint interval automatically set to " + checkpointDuration)
+ }
+
+ // Set the minimum value of the rememberDuration if not already set
+ var minRememberDuration = slideDuration
+ if (checkpointDuration != null && minRememberDuration <= checkpointDuration) {
+ minRememberDuration = checkpointDuration * 2 // times 2 just to be sure that the latest checkpoint is not forgetten
+ }
+ if (rememberDuration == null || rememberDuration < minRememberDuration) {
+ rememberDuration = minRememberDuration
+ }
+
+ // Initialize the dependencies
+ dependencies.foreach(_.initialize(zeroTime))
+ }
+
+ protected[streaming] def validate() {
+ assert(rememberDuration != null, "Remember duration is set to null")
+
+ assert(
+ !mustCheckpoint || checkpointDuration != null,
+ "The checkpoint interval for " + this.getClass.getSimpleName + " has not been set." +
+ " Please use DStream.checkpoint() to set the interval."
+ )
+
+ assert(
+ checkpointDuration == null || ssc.sc.checkpointDir.isDefined,
+ "The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" +
+ " or SparkContext.checkpoint() to set the checkpoint directory."
+ )
+
+ assert(
+ checkpointDuration == null || checkpointDuration >= slideDuration,
+ "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " +
+ checkpointDuration + " which is lower than its slide time (" + slideDuration + "). " +
+ "Please set it to at least " + slideDuration + "."
+ )
+
+ assert(
+ checkpointDuration == null || checkpointDuration.isMultipleOf(slideDuration),
+ "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " +
+ checkpointDuration + " which not a multiple of its slide time (" + slideDuration + "). " +
+ "Please set it to a multiple " + slideDuration + "."
+ )
+
+ assert(
+ checkpointDuration == null || storageLevel != StorageLevel.NONE,
+ "" + this.getClass.getSimpleName + " has been marked for checkpointing but the storage " +
+ "level has not been set to enable persisting. Please use DStream.persist() to set the " +
+ "storage level to use memory for better checkpointing performance."
+ )
+
+ assert(
+ checkpointDuration == null || rememberDuration > checkpointDuration,
+ "The remember duration for " + this.getClass.getSimpleName + " has been set to " +
+ rememberDuration + " which is not more than the checkpoint interval (" +
+ checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
+ )
+
+ val metadataCleanerDelay = spark.util.MetadataCleaner.getDelaySeconds
+ logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
+ assert(
+ metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
+ "It seems you are doing some DStream window operation or setting a checkpoint interval " +
+ "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " +
+ "than " + rememberDuration.milliseconds + " milliseconds. But the Spark's metadata cleanup" +
+ "delay is set to " + (metadataCleanerDelay / 60.0) + " minutes, which is not sufficient. Please set " +
+ "the Java property 'spark.cleaner.delay' to more than " +
+ math.ceil(rememberDuration.milliseconds.toDouble / 60000.0).toInt + " minutes."
+ )
+
+ dependencies.foreach(_.validate())
+
+ logInfo("Slide time = " + slideDuration)
+ logInfo("Storage level = " + storageLevel)
+ logInfo("Checkpoint interval = " + checkpointDuration)
+ logInfo("Remember duration = " + rememberDuration)
+ logInfo("Initialized and validated " + this)
+ }
+
+ protected[streaming] def setContext(s: StreamingContext) {
+ if (ssc != null && ssc != s) {
+ throw new Exception("Context is already set in " + this + ", cannot set it again")
+ }
+ ssc = s
+ logInfo("Set context for " + this)
+ dependencies.foreach(_.setContext(ssc))
+ }
+
+ protected[streaming] def setGraph(g: DStreamGraph) {
+ if (graph != null && graph != g) {
+ throw new Exception("Graph is already set in " + this + ", cannot set it again")
+ }
+ graph = g
+ dependencies.foreach(_.setGraph(graph))
+ }
+
+ protected[streaming] def remember(duration: Duration) {
+ if (duration != null && duration > rememberDuration) {
+ rememberDuration = duration
+ logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this)
+ }
+ dependencies.foreach(_.remember(parentRememberDuration))
+ }
+
+ /** This method checks whether the 'time' is valid wrt slideDuration for generating RDD */
+ protected def isTimeValid(time: Time): Boolean = {
+ if (!isInitialized) {
+ throw new Exception (this + " has not been initialized")
+ } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) {
+ false
+ } else {
+ true
+ }
+ }
+
+ /**
+ * Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal
+ * method that should not be called directly.
+ */
+ protected[streaming] def getOrCompute(time: Time): Option[RDD[T]] = {
+ // If this DStream was not initialized (i.e., zeroTime not set), then do it
+ // If RDD was already generated, then retrieve it from HashMap
+ generatedRDDs.get(time) match {
+
+ // If an RDD was already generated and is being reused, then
+ // probably all RDDs in this DStream will be reused and hence should be cached
+ case Some(oldRDD) => Some(oldRDD)
+
+ // if RDD was not generated, and if the time is valid
+ // (based on sliding time of this DStream), then generate the RDD
+ case None => {
+ if (isTimeValid(time)) {
+ compute(time) match {
+ case Some(newRDD) =>
+ if (storageLevel != StorageLevel.NONE) {
+ newRDD.persist(storageLevel)
+ logInfo("Persisting RDD " + newRDD.id + " for time " + time + " to " + storageLevel + " at time " + time)
+ }
+ if (checkpointDuration != null && (time - zeroTime).isMultipleOf(checkpointDuration)) {
+ newRDD.checkpoint()
+ logInfo("Marking RDD " + newRDD.id + " for time " + time + " for checkpointing at time " + time)
+ }
+ generatedRDDs.put(time, newRDD)
+ Some(newRDD)
+ case None =>
+ None
+ }
+ } else {
+ None
+ }
+ }
+ }
+ }
+
+ /**
+ * 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
+ * that materializes the corresponding RDD. Subclasses of DStream may override this
+ * (eg. ForEachDStream).
+ */
+ protected[streaming] def generateJob(time: Time): Option[Job] = {
+ getOrCompute(time) match {
+ case Some(rdd) => {
+ val jobFunc = () => {
+ val emptyFunc = { (iterator: Iterator[T]) => {} }
+ ssc.sc.runJob(rdd, emptyFunc)
+ }
+ Some(new Job(time, jobFunc))
+ }
+ case None => None
+ }
+ }
+
+ /**
+ * Dereference RDDs that are older than rememberDuration.
+ */
+ protected[streaming] def forgetOldRDDs(time: Time) {
+ val keys = generatedRDDs.keys
+ var numForgotten = 0
+ keys.foreach(t => {
+ if (t <= (time - rememberDuration)) {
+ generatedRDDs.remove(t)
+ numForgotten += 1
+ logInfo("Forgot RDD of time " + t + " from " + this)
+ }
+ })
+ logInfo("Forgot " + numForgotten + " RDDs from " + this)
+ dependencies.foreach(_.forgetOldRDDs(time))
+ }
+
+ /* Adds metadata to the Stream while it is running.
+ * This methd should be overwritten by sublcasses of InputDStream.
+ */
+ protected[streaming] def addMetadata(metadata: Any) {
+ if (metadata != null) {
+ logInfo("Dropping Metadata: " + metadata.toString)
+ }
+ }
+
+ /**
+ * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of
+ * this stream. This is an internal method that should not be called directly. This is
+ * a default implementation that saves only the file names of the checkpointed RDDs to
+ * checkpointData. Subclasses of DStream (especially those of InputDStream) may override
+ * this method to save custom checkpoint data.
+ */
+ protected[streaming] def updateCheckpointData(currentTime: Time) {
+ logInfo("Updating checkpoint data for time " + currentTime)
+
+ // 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))
+
+ // TODO: remove this, this is just for debugging
+ 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(",") + "]")
+ }
+
+ /**
+ * Restore the RDDs in generatedRDDs from the checkpointData. This is an internal method
+ * that should not be called directly. This is a default implementation that recreates RDDs
+ * from the checkpoint file names stored in checkpointData. Subclasses of DStream that
+ * override the updateCheckpointData() method would also need to override this method.
+ */
+ protected[streaming] def restoreCheckpointData() {
+ // Create RDDs from the checkpoint data
+ logInfo("Restoring checkpoint data from " + checkpointData.rdds.size + " checkpointed RDDs")
+ checkpointData.rdds.foreach {
+ 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())
+ logInfo("Restored checkpoint data")
+ }
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ logDebug(this.getClass().getSimpleName + ".writeObject used")
+ if (graph != null) {
+ graph.synchronized {
+ if (graph.checkpointInProgress) {
+ oos.defaultWriteObject()
+ } else {
+ val msg = "Object of " + this.getClass.getName + " is being serialized " +
+ " possibly as a part of closure of an RDD operation. This is because " +
+ " the DStream object is being referred to from within the closure. " +
+ " Please rewrite the RDD operation inside this DStream to avoid this. " +
+ " This has been enforced to avoid bloating of Spark tasks " +
+ " with unnecessary objects."
+ throw new java.io.NotSerializableException(msg)
+ }
+ }
+ } else {
+ throw new java.io.NotSerializableException("Graph is unexpectedly null when DStream is being serialized.")
+ }
+ }
+
+ @throws(classOf[IOException])
+ private def readObject(ois: ObjectInputStream) {
+ logDebug(this.getClass().getSimpleName + ".readObject used")
+ ois.defaultReadObject()
+ generatedRDDs = new HashMap[Time, RDD[T]] ()
+ }
+
+ // =======================================================================
+ // DStream operations
+ // =======================================================================
+
+ /** Return a new DStream by applying a function to all elements of this DStream. */
+ def map[U: ClassManifest](mapFunc: T => U): DStream[U] = {
+ new MappedDStream(this, ssc.sc.clean(mapFunc))
+ }
+
+ /**
+ * Return a new DStream by applying a function to all elements of this DStream,
+ * and then flattening the results
+ */
+ def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
+ new FlatMappedDStream(this, ssc.sc.clean(flatMapFunc))
+ }
+
+ /** Return a new DStream containing only the elements that satisfy a predicate. */
+ def filter(filterFunc: T => Boolean): DStream[T] = new FilteredDStream(this, filterFunc)
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying glom() to each RDD of
+ * this DStream. Applying glom() to an RDD coalesces all elements within each partition into
+ * an array.
+ */
+ def glom(): DStream[Array[T]] = new GlommedDStream(this)
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+ * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+ * of the RDD.
+ */
+ def mapPartitions[U: ClassManifest](
+ mapPartFunc: Iterator[T] => Iterator[U],
+ preservePartitioning: Boolean = false
+ ): DStream[U] = {
+ new MapPartitionedDStream(this, ssc.sc.clean(mapPartFunc), preservePartitioning)
+ }
+
+ /**
+ * Return a new DStream in which each RDD has a single element generated by reducing each RDD
+ * of this DStream.
+ */
+ def reduce(reduceFunc: (T, T) => T): DStream[T] =
+ this.map(x => (null, x)).reduceByKey(reduceFunc, 1).map(_._2)
+
+ /**
+ * Return a new DStream in which each RDD has a single element generated by counting each RDD
+ * of this DStream.
+ */
+ def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _)
+
+ /**
+ * 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.
+ */
+ def foreach(foreachFunc: RDD[T] => Unit) {
+ foreach((r: RDD[T], t: Time) => foreachFunc(r))
+ }
+
+ /**
+ * 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.
+ */
+ def foreach(foreachFunc: (RDD[T], Time) => Unit) {
+ val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc))
+ ssc.registerOutputStream(newStream)
+ newStream
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying a function
+ * on each RDD of this DStream.
+ */
+ def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
+ transform((r: RDD[T], t: Time) => transformFunc(r))
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying a function
+ * on each RDD of this DStream.
+ */
+ def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
+ new TransformedDStream(this, ssc.sc.clean(transformFunc))
+ }
+
+ /**
+ * Print the first ten elements of each RDD generated in this DStream. This is an output
+ * operator, so this DStream will be registered as an output stream and there materialized.
+ */
+ def print() {
+ def foreachFunc = (rdd: RDD[T], time: Time) => {
+ val first11 = rdd.take(11)
+ println ("-------------------------------------------")
+ println ("Time: " + time)
+ println ("-------------------------------------------")
+ first11.take(10).foreach(println)
+ if (first11.size > 10) println("...")
+ println()
+ }
+ val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc))
+ ssc.registerOutputStream(newStream)
+ }
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of 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.
+ */
+ def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration)
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of this DStream.
+ * @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 window(windowDuration: Duration, slideDuration: Duration): DStream[T] = {
+ 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
+ * elements in a window over this DStream. windowDuration and slideDuration are as defined
+ * in the window() operation. This is equivalent to
+ * window(windowDuration, slideDuration).reduce(reduceFunc)
+ */
+ def reduceByWindow(
+ reduceFunc: (T, T) => T,
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): DStream[T] = {
+ this.window(windowDuration, slideDuration).reduce(reduceFunc)
+ }
+
+ def reduceByWindow(
+ reduceFunc: (T, T) => T,
+ invReduceFunc: (T, T) => T,
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): DStream[T] = {
+ this.map(x => (1, x))
+ .reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1)
+ .map(_._2)
+ }
+
+ /**
+ * 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
+ * window() operation. This is equivalent to window(windowDuration, slideDuration).count()
+ */
+ def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = {
+ this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration)
+ }
+
+ /**
+ * Return a new DStream by unifying data of another DStream with this DStream.
+ * @param that Another DStream having the same slideDuration as this DStream.
+ */
+ def union(that: DStream[T]): DStream[T] = new UnionDStream[T](Array(this, that))
+
+ /**
+ * Return all the RDDs defined by the Interval object (both end times included)
+ */
+ protected[streaming] def slice(interval: Interval): Seq[RDD[T]] = {
+ slice(interval.beginTime, interval.endTime)
+ }
+
+ /**
+ * Return all the RDDs between 'fromTime' to 'toTime' (both included)
+ */
+ def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = {
+ val rdds = new ArrayBuffer[RDD[T]]()
+ var time = toTime.floor(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
+ }
+
+ /**
+ * Save each RDD in this DStream as a Sequence file of serialized objects.
+ * The file name at each batch interval is generated based on `prefix` and
+ * `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsObjectFiles(prefix: String, suffix: String = "") {
+ val saveFunc = (rdd: RDD[T], time: Time) => {
+ val file = rddToFileName(prefix, suffix, time)
+ rdd.saveAsObjectFile(file)
+ }
+ this.foreach(saveFunc)
+ }
+
+ /**
+ * Save each RDD in this DStream as at text file, using string representation
+ * of elements. The file name at each batch interval is generated based on
+ * `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsTextFiles(prefix: String, suffix: String = "") {
+ val saveFunc = (rdd: RDD[T], time: Time) => {
+ val file = rddToFileName(prefix, suffix, time)
+ rdd.saveAsTextFile(file)
+ }
+ this.foreach(saveFunc)
+ }
+
+ def register() {
+ ssc.registerOutputStream(this)
+ }
+}
+
+private[streaming]
+case class DStreamCheckpointData(rdds: HashMap[Time, Any])
+
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
new file mode 100644
index 0000000000..bc4a40d7bc
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -0,0 +1,134 @@
+package spark.streaming
+
+import dstream.InputDStream
+import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
+import collection.mutable.ArrayBuffer
+import spark.Logging
+
+final private[streaming] class DStreamGraph extends Serializable with Logging {
+ initLogging()
+
+ private val inputStreams = new ArrayBuffer[InputDStream[_]]()
+ private val outputStreams = new ArrayBuffer[DStream[_]]()
+
+ private[streaming] var zeroTime: Time = null
+ private[streaming] var batchDuration: Duration = null
+ private[streaming] var rememberDuration: Duration = null
+ private[streaming] var checkpointInProgress = false
+
+ private[streaming] def start(time: Time) {
+ this.synchronized {
+ if (zeroTime != null) {
+ throw new Exception("DStream graph computation already started")
+ }
+ zeroTime = time
+ outputStreams.foreach(_.initialize(zeroTime))
+ outputStreams.foreach(_.remember(rememberDuration))
+ outputStreams.foreach(_.validate)
+ inputStreams.par.foreach(_.start())
+ }
+ }
+
+ private[streaming] def stop() {
+ this.synchronized {
+ inputStreams.par.foreach(_.stop())
+ }
+ }
+
+ private[streaming] def setContext(ssc: StreamingContext) {
+ this.synchronized {
+ outputStreams.foreach(_.setContext(ssc))
+ }
+ }
+
+ private[streaming] def setBatchDuration(duration: Duration) {
+ this.synchronized {
+ if (batchDuration != null) {
+ throw new Exception("Batch duration already set as " + batchDuration +
+ ". cannot set it again.")
+ }
+ }
+ batchDuration = duration
+ }
+
+ private[streaming] def remember(duration: Duration) {
+ this.synchronized {
+ if (rememberDuration != null) {
+ throw new Exception("Batch duration already set as " + batchDuration +
+ ". cannot set it again.")
+ }
+ }
+ rememberDuration = duration
+ }
+
+ private[streaming] def addInputStream(inputStream: InputDStream[_]) {
+ this.synchronized {
+ inputStream.setGraph(this)
+ inputStreams += inputStream
+ }
+ }
+
+ private[streaming] def addOutputStream(outputStream: DStream[_]) {
+ this.synchronized {
+ outputStream.setGraph(this)
+ outputStreams += outputStream
+ }
+ }
+
+ private[streaming] def getInputStreams() = this.synchronized { inputStreams.toArray }
+
+ private[streaming] def getOutputStreams() = this.synchronized { outputStreams.toArray }
+
+ private[streaming] def generateRDDs(time: Time): Seq[Job] = {
+ this.synchronized {
+ outputStreams.flatMap(outputStream => outputStream.generateJob(time))
+ }
+ }
+
+ private[streaming] def forgetOldRDDs(time: Time) {
+ this.synchronized {
+ outputStreams.foreach(_.forgetOldRDDs(time))
+ }
+ }
+
+ private[streaming] def updateCheckpointData(time: Time) {
+ this.synchronized {
+ outputStreams.foreach(_.updateCheckpointData(time))
+ }
+ }
+
+ private[streaming] def restoreCheckpointData() {
+ this.synchronized {
+ outputStreams.foreach(_.restoreCheckpointData())
+ }
+ }
+
+ private[streaming] def validate() {
+ this.synchronized {
+ assert(batchDuration != null, "Batch duration has not been set")
+ //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low")
+ assert(getOutputStreams().size > 0, "No output streams registered, so nothing to execute")
+ }
+ }
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ this.synchronized {
+ logDebug("DStreamGraph.writeObject used")
+ checkpointInProgress = true
+ oos.defaultWriteObject()
+ checkpointInProgress = false
+ }
+ }
+
+ @throws(classOf[IOException])
+ private def readObject(ois: ObjectInputStream) {
+ this.synchronized {
+ logDebug("DStreamGraph.readObject used")
+ checkpointInProgress = true
+ ois.defaultReadObject()
+ checkpointInProgress = false
+ }
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala
new file mode 100644
index 0000000000..e4dc579a17
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Duration.scala
@@ -0,0 +1,62 @@
+package spark.streaming
+
+case class Duration (private val millis: Long) {
+
+ def < (that: Duration): Boolean = (this.millis < that.millis)
+
+ def <= (that: Duration): Boolean = (this.millis <= that.millis)
+
+ def > (that: Duration): Boolean = (this.millis > that.millis)
+
+ def >= (that: Duration): Boolean = (this.millis >= that.millis)
+
+ def + (that: Duration): Duration = new Duration(millis + that.millis)
+
+ def - (that: Duration): Duration = new Duration(millis - that.millis)
+
+ def * (times: Int): Duration = new Duration(millis * times)
+
+ def / (that: Duration): Long = millis / that.millis
+
+ def isMultipleOf(that: Duration): Boolean =
+ (this.millis % that.millis == 0)
+
+ def min(that: Duration): Duration = if (this < that) this else that
+
+ def max(that: Duration): Duration = if (this > that) this else that
+
+ def isZero: Boolean = (this.millis == 0)
+
+ override def toString: String = (millis.toString + " ms")
+
+ def toFormattedString: String = millis.toString
+
+ def milliseconds: Long = millis
+}
+
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of milliseconds.
+ */
+object Milliseconds {
+ def apply(milliseconds: Long) = new Duration(milliseconds)
+}
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of seconds.
+ */
+object Seconds {
+ def apply(seconds: Long) = new Duration(seconds * 1000)
+}
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of minutes.
+ */
+object Minutes {
+ def apply(minutes: Long) = new Duration(minutes * 60000)
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala
new file mode 100644
index 0000000000..dc21dfb722
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Interval.scala
@@ -0,0 +1,41 @@
+package spark.streaming
+
+private[streaming]
+class Interval(val beginTime: Time, val endTime: Time) {
+ def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs))
+
+ def duration(): Duration = endTime - beginTime
+
+ def + (time: Duration): Interval = {
+ new Interval(beginTime + time, endTime + time)
+ }
+
+ def - (time: Duration): Interval = {
+ new Interval(beginTime - time, endTime - time)
+ }
+
+ def < (that: Interval): Boolean = {
+ if (this.duration != that.duration) {
+ throw new Exception("Comparing two intervals with different durations [" + this + ", " + that + "]")
+ }
+ this.endTime < that.endTime
+ }
+
+ def <= (that: Interval) = (this < that || this == that)
+
+ def > (that: Interval) = !(this <= that)
+
+ def >= (that: Interval) = !(this < that)
+
+ override def toString = "[" + beginTime + ", " + endTime + "]"
+}
+
+object Interval {
+ def currentInterval(duration: Duration): Interval = {
+ val time = new Time(System.currentTimeMillis)
+ val intervalBegin = time.floor(duration)
+ new Interval(intervalBegin, intervalBegin + duration)
+ }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala
new file mode 100644
index 0000000000..67bd8388bc
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Job.scala
@@ -0,0 +1,24 @@
+package spark.streaming
+
+import java.util.concurrent.atomic.AtomicLong
+
+private[streaming]
+class Job(val time: Time, func: () => _) {
+ val id = Job.getNewId()
+ def run(): Long = {
+ val startTime = System.currentTimeMillis
+ func()
+ val stopTime = System.currentTimeMillis
+ (stopTime - startTime)
+ }
+
+ override def toString = "streaming job " + id + " @ " + time
+}
+
+private[streaming]
+object Job {
+ val id = new AtomicLong(0)
+
+ def getNewId() = id.getAndIncrement()
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala
new file mode 100644
index 0000000000..3b910538e0
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/JobManager.scala
@@ -0,0 +1,33 @@
+package spark.streaming
+
+import spark.Logging
+import spark.SparkEnv
+import java.util.concurrent.Executors
+
+
+private[streaming]
+class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging {
+
+ class JobHandler(ssc: StreamingContext, job: Job) extends Runnable {
+ def run() {
+ SparkEnv.set(ssc.env)
+ try {
+ val timeTaken = job.run()
+ logInfo("Total delay: %.5f s for job %s (execution: %.5f s)".format(
+ (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, timeTaken / 1000.0))
+ } catch {
+ case e: Exception =>
+ logError("Running " + job + " failed", e)
+ }
+ }
+ }
+
+ initLogging()
+
+ val jobExecutor = Executors.newFixedThreadPool(numThreads)
+
+ def runJob(job: Job) {
+ jobExecutor.execute(new JobHandler(ssc, job))
+ logInfo("Added " + job + " to queue")
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
new file mode 100644
index 0000000000..e4152f3a61
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
@@ -0,0 +1,151 @@
+package spark.streaming
+
+import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
+import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
+import spark.Logging
+import spark.SparkEnv
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Queue
+
+import akka.actor._
+import akka.pattern.ask
+import akka.util.duration._
+import akka.dispatch._
+
+private[streaming] sealed trait NetworkInputTrackerMessage
+private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
+private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[String], metadata: Any) extends NetworkInputTrackerMessage
+private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage
+
+/**
+ * This class manages the execution of the receivers of NetworkInputDStreams.
+ */
+private[streaming]
+class NetworkInputTracker(
+ @transient ssc: StreamingContext,
+ @transient networkInputStreams: Array[NetworkInputDStream[_]])
+ extends Logging {
+
+ val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*)
+ val receiverExecutor = new ReceiverExecutor()
+ val receiverInfo = new HashMap[Int, ActorRef]
+ val receivedBlockIds = new HashMap[Int, Queue[String]]
+ val timeout = 5000.milliseconds
+
+ var currentTime: Time = null
+
+ /** Start the actor and receiver execution thread. */
+ def start() {
+ ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), "NetworkInputTracker")
+ receiverExecutor.start()
+ }
+
+ /** Stop the receiver execution thread. */
+ def stop() {
+ // TODO: stop the actor as well
+ receiverExecutor.interrupt()
+ receiverExecutor.stopReceivers()
+ }
+
+ /** Return all the blocks received from a receiver. */
+ def getBlockIds(receiverId: Int, time: Time): Array[String] = synchronized {
+ val queue = receivedBlockIds.synchronized {
+ receivedBlockIds.getOrElse(receiverId, new Queue[String]())
+ }
+ val result = queue.synchronized {
+ queue.dequeueAll(x => true)
+ }
+ logInfo("Stream " + receiverId + " received " + result.size + " blocks")
+ result.toArray
+ }
+
+ /** Actor to receive messages from the receivers. */
+ private class NetworkInputTrackerActor extends Actor {
+ def receive = {
+ case RegisterReceiver(streamId, receiverActor) => {
+ if (!networkInputStreamMap.contains(streamId)) {
+ throw new Exception("Register received for unexpected id " + streamId)
+ }
+ receiverInfo += ((streamId, receiverActor))
+ logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address)
+ sender ! true
+ }
+ case AddBlocks(streamId, blockIds, metadata) => {
+ val tmp = receivedBlockIds.synchronized {
+ if (!receivedBlockIds.contains(streamId)) {
+ receivedBlockIds += ((streamId, new Queue[String]))
+ }
+ receivedBlockIds(streamId)
+ }
+ tmp.synchronized {
+ tmp ++= blockIds
+ }
+ networkInputStreamMap(streamId).addMetadata(metadata)
+ }
+ case DeregisterReceiver(streamId, msg) => {
+ receiverInfo -= streamId
+ logInfo("De-registered receiver for network stream " + streamId
+ + " with message " + msg)
+ //TODO: Do something about the corresponding NetworkInputDStream
+ }
+ }
+ }
+
+ /** This thread class runs all the receivers on the cluster. */
+ class ReceiverExecutor extends Thread {
+ val env = ssc.env
+
+ override def run() {
+ try {
+ SparkEnv.set(env)
+ startReceivers()
+ } catch {
+ case ie: InterruptedException => logInfo("ReceiverExecutor interrupted")
+ } finally {
+ stopReceivers()
+ }
+ }
+
+ /**
+ * Get the receivers from the NetworkInputDStreams, distributes them to the
+ * worker nodes as a parallel collection, and runs them.
+ */
+ def startReceivers() {
+ val receivers = networkInputStreams.map(nis => {
+ val rcvr = nis.createReceiver()
+ rcvr.setStreamId(nis.id)
+ rcvr
+ })
+
+ // Right now, we only honor preferences if all receivers have them
+ val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined).reduce(_ && _)
+
+ // Create the parallel collection of receivers to distributed them on the worker nodes
+ val tempRDD =
+ if (hasLocationPreferences) {
+ val receiversWithPreferences = receivers.map(r => (r, Seq(r.getLocationPreference().toString)))
+ ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences)
+ }
+ else {
+ ssc.sc.makeRDD(receivers, receivers.size)
+ }
+
+ // Function to start the receiver on the worker node
+ val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => {
+ if (!iterator.hasNext) {
+ throw new Exception("Could not start receiver as details not found.")
+ }
+ iterator.next().start()
+ }
+ // Distribute the receivers and start them
+ ssc.sc.runJob(tempRDD, startReceiver)
+ }
+
+ /** Stops the receivers. */
+ def stopReceivers() {
+ // Signal the receivers to stop
+ receiverInfo.values.foreach(_ ! StopReceiver)
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
new file mode 100644
index 0000000000..fbcf061126
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
@@ -0,0 +1,562 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
+import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream}
+import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
+
+import spark.{Manifests, RDD, Partitioner, HashPartitioner}
+import spark.SparkContext._
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.mapred.{JobConf, OutputFormat}
+import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
+import org.apache.hadoop.mapred.OutputFormat
+import org.apache.hadoop.conf.Configuration
+
+class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)])
+extends Serializable {
+
+ def ssc = self.ssc
+
+ private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = {
+ new HashPartitioner(numPartitions)
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+ * generate the RDDs with Spark's default number of partitions.
+ */
+ def groupByKey(): DStream[(K, Seq[V])] = {
+ groupByKey(defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+ * generate the RDDs with `numPartitions` partitions.
+ */
+ def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = {
+ groupByKey(defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]]
+ * is used to control the partitioning of each RDD.
+ */
+ def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = {
+ val createCombiner = (v: V) => ArrayBuffer[V](v)
+ val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v)
+ val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2)
+ combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner)
+ .asInstanceOf[DStream[(K, Seq[V])]]
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the associative reduce function. Hash partitioning is used to generate the RDDs
+ * with Spark's default number of partitions.
+ */
+ def reduceByKey(reduceFunc: (V, V) => V): DStream[(K, V)] = {
+ reduceByKey(reduceFunc, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs
+ * with `numPartitions` partitions.
+ */
+ def reduceByKey(reduceFunc: (V, V) => V, numPartitions: Int): DStream[(K, V)] = {
+ reduceByKey(reduceFunc, defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+ * partitioning of each RDD.
+ */
+ def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = {
+ val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+ combineByKey((v: V) => v, cleanedReduceFunc, cleanedReduceFunc, partitioner)
+ }
+
+ /**
+ * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+ * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+ * information.
+ */
+ def combineByKey[C: ClassManifest](
+ createCombiner: V => C,
+ mergeValue: (C, V) => C,
+ mergeCombiner: (C, C) => C,
+ partitioner: Partitioner) : DStream[(K, C)] = {
+ new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
+ }
+
+ /**
+ * Create a new DStream by counting the number of values of each key in each RDD. Hash
+ * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions.
+ */
+ def countByKey(numPartitions: Int = self.ssc.sc.defaultParallelism): DStream[(K, Long)] = {
+ self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
+ }
+
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to
+ * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
+ * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
+ * Spark's default number of partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
+ def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = {
+ groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window. Similar to
+ * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with 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 groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = {
+ groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def groupByKeyAndWindow(
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int
+ ): DStream[(K, Seq[V])] = {
+ groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+ * @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 partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def groupByKeyAndWindow(
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ): DStream[(K, Seq[V])] = {
+ self.window(windowDuration, slideDuration).groupByKey(partitioner)
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream
+ * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate
+ * the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ windowDuration: Duration
+ ): DStream[(K, V)] = {
+ reduceByKeyAndWindow(reduceFunc, windowDuration, self.slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+ * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative 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 reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): DStream[(K, V)] = {
+ reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+ * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int
+ ): DStream[(K, V)] = {
+ reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
+ * `DStream.reduceByKey()`, but applies it over a sliding window.
+ * @param reduceFunc associative 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
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ): DStream[(K, V)] = {
+ val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+ self.reduceByKey(cleanedReduceFunc, partitioner)
+ .window(windowDuration, slideDuration)
+ .reduceByKey(cleanedReduceFunc, partitioner)
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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 reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ invReduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): DStream[(K, V)] = {
+
+ reduceByKeyAndWindow(
+ reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ invReduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int
+ ): DStream[(K, V)] = {
+
+ reduceByKeyAndWindow(
+ reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: (V, V) => V,
+ invReduceFunc: (V, V) => V,
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ): DStream[(K, V)] = {
+
+ val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+ val cleanedInvReduceFunc = ssc.sc.clean(invReduceFunc)
+ new ReducedWindowedDStream[K, V](
+ self, cleanedReduceFunc, cleanedInvReduceFunc, windowDuration, slideDuration, partitioner)
+ }
+
+ /**
+ * Create a new DStream by counting the number of values for each key over a window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def countByKeyAndWindow(
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int = self.ssc.sc.defaultParallelism
+ ): DStream[(K, Long)] = {
+
+ self.map(x => (x._1, 1L)).reduceByKeyAndWindow(
+ (x: Long, y: Long) => x + y,
+ (x: Long, y: Long) => x - y,
+ windowDuration,
+ slideDuration,
+ numPartitions
+ )
+ }
+
+ /**
+ * Create 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.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: (Seq[V], Option[S]) => Option[S]
+ ): DStream[(K, S)] = {
+ updateStateByKey(updateFunc, defaultPartitioner())
+ }
+
+ /**
+ * Create 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.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: (Seq[V], Option[S]) => Option[S],
+ numPartitions: Int
+ ): DStream[(K, S)] = {
+ updateStateByKey(updateFunc, defaultPartitioner(numPartitions))
+ }
+
+ /**
+ * Create 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 the key.
+ * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: (Seq[V], Option[S]) => Option[S],
+ partitioner: Partitioner
+ ): DStream[(K, S)] = {
+ val newUpdateFunc = (iterator: Iterator[(K, Seq[V], Option[S])]) => {
+ iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s)))
+ }
+ updateStateByKey(newUpdateFunc, partitioner, true)
+ }
+
+ /**
+ * Create 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.
+ * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated. Note, that
+ * this function may generate a different a tuple with a different key
+ * than the input key. It is up to the developer to decide whether to
+ * remember the partitioner despite the key being changed.
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
+ partitioner: Partitioner,
+ rememberPartitioner: Boolean
+ ): DStream[(K, S)] = {
+ new StateDStream(self, ssc.sc.clean(updateFunc), partitioner, rememberPartitioner)
+ }
+
+
+ def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = {
+ new MapValuedDStream[K, V, U](self, mapValuesFunc)
+ }
+
+ def flatMapValues[U: ClassManifest](
+ flatMapValuesFunc: V => TraversableOnce[U]
+ ): DStream[(K, U)] = {
+ new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
+ }
+
+ /**
+ * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+ * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+ * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+ * of partitions.
+ */
+ def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
+ cogroup(other, defaultPartitioner())
+ }
+
+ /**
+ * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+ * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+ * key in both RDDs. Partitioner is used to partition each generated RDD.
+ */
+ def cogroup[W: ClassManifest](
+ other: DStream[(K, W)],
+ partitioner: Partitioner
+ ): DStream[(K, (Seq[V], Seq[W]))] = {
+
+ val cgd = new CoGroupedDStream[K](
+ Seq(self.asInstanceOf[DStream[(_, _)]], other.asInstanceOf[DStream[(_, _)]]),
+ partitioner
+ )
+ val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)(
+ classManifest[K],
+ Manifests.seqSeqManifest
+ )
+ pdfs.mapValues {
+ case Seq(vs, ws) =>
+ (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
+ }
+ }
+
+ /**
+ * Join `this` DStream with `other` DStream. HashPartitioner is used
+ * to partition each generated RDD into default number of partitions.
+ */
+ def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
+ join[W](other, defaultPartitioner())
+ }
+
+ /**
+ * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
+ * be generated by joining RDDs from `this` and other DStream. Uses the given
+ * Partitioner to partition each generated RDD.
+ */
+ def join[W: ClassManifest](
+ other: DStream[(K, W)],
+ partitioner: Partitioner
+ ): DStream[(K, (V, W))] = {
+ this.cogroup(other, partitioner)
+ .flatMapValues{
+ case (vs, ws) =>
+ for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
+ }
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+ */
+ def saveAsHadoopFiles[F <: OutputFormat[K, V]](
+ prefix: String,
+ suffix: String
+ )(implicit fm: ClassManifest[F]) {
+ saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+ */
+ def saveAsHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: OutputFormat[_, _]],
+ conf: JobConf = new JobConf
+ ) {
+ val saveFunc = (rdd: RDD[(K, V)], time: Time) => {
+ val file = rddToFileName(prefix, suffix, time)
+ rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
+ }
+ self.foreach(saveFunc)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](
+ prefix: String,
+ suffix: String
+ )(implicit fm: ClassManifest[F]) {
+ saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsNewAPIHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
+ conf: Configuration = new Configuration
+ ) {
+ val saveFunc = (rdd: RDD[(K, V)], time: Time) => {
+ val file = rddToFileName(prefix, suffix, time)
+ rdd.saveAsNewAPIHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
+ }
+ self.foreach(saveFunc)
+ }
+
+ private def getKeyClass() = implicitly[ClassManifest[K]].erasure
+
+ private def getValueClass() = implicitly[ClassManifest[V]].erasure
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala
new file mode 100644
index 0000000000..c04ed37de8
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala
@@ -0,0 +1,77 @@
+package spark.streaming
+
+import util.{ManualClock, RecurringTimer, Clock}
+import spark.SparkEnv
+import spark.Logging
+
+private[streaming]
+class Scheduler(ssc: StreamingContext) extends Logging {
+
+ initLogging()
+
+ val graph = ssc.graph
+
+ val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
+ val jobManager = new JobManager(ssc, concurrentJobs)
+
+ val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
+ new CheckpointWriter(ssc.checkpointDir)
+ } else {
+ null
+ }
+
+ val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock")
+ val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+ val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
+ longTime => generateRDDs(new Time(longTime)))
+
+ def start() {
+ // If context was started from checkpoint, then restart timer such that
+ // this timer's triggers occur at the same time as the original timer.
+ // Otherwise just start the timer from scratch, and initialize graph based
+ // on this first trigger time of the timer.
+ if (ssc.isCheckpointPresent) {
+ // If manual clock is being used for testing, then
+ // either set the manual clock to the last checkpointed time,
+ // or if the property is defined set it to that time
+ if (clock.isInstanceOf[ManualClock]) {
+ val lastTime = ssc.getInitialCheckpoint.checkpointTime.milliseconds
+ val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
+ clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
+ }
+ timer.restart(graph.zeroTime.milliseconds)
+ logInfo("Scheduler's timer restarted")
+ } else {
+ val firstTime = new Time(timer.start())
+ graph.start(firstTime - ssc.graph.batchDuration)
+ logInfo("Scheduler's timer started")
+ }
+ logInfo("Scheduler started")
+ }
+
+ def stop() {
+ timer.stop()
+ graph.stop()
+ logInfo("Scheduler stopped")
+ }
+
+ private def generateRDDs(time: Time) {
+ SparkEnv.set(ssc.env)
+ logInfo("\n-----------------------------------------------------\n")
+ graph.generateRDDs(time).foreach(jobManager.runJob)
+ graph.forgetOldRDDs(time)
+ doCheckpoint(time)
+ logInfo("Generated RDDs for time " + time)
+ }
+
+ private def doCheckpoint(time: Time) {
+ if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
+ val startTime = System.currentTimeMillis()
+ ssc.graph.updateCheckpointData(time)
+ checkpointWriter.write(new Checkpoint(ssc, time))
+ val stopTime = System.currentTimeMillis()
+ logInfo("Checkpointing the graph took " + (stopTime - startTime) + " ms")
+ }
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
new file mode 100644
index 0000000000..14500bdcb1
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -0,0 +1,411 @@
+package spark.streaming
+
+import spark.streaming.dstream._
+
+import spark.{RDD, Logging, SparkEnv, SparkContext}
+import spark.storage.StorageLevel
+import spark.util.MetadataCleaner
+
+import scala.collection.mutable.Queue
+
+import java.io.InputStream
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.hadoop.io.LongWritable
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+import org.apache.hadoop.fs.Path
+import java.util.UUID
+
+/**
+ * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
+ * information (such as, cluster URL and job name) to internally create a SparkContext, it provides
+ * methods used to create DStream from various input sources.
+ */
+class StreamingContext private (
+ sc_ : SparkContext,
+ cp_ : Checkpoint,
+ batchDur_ : Duration
+ ) extends Logging {
+
+ /**
+ * Creates a StreamingContext using an existing SparkContext.
+ * @param sparkContext Existing SparkContext
+ * @param batchDuration The time interval at which streaming data will be divided into batches
+ */
+ def this(sparkContext: SparkContext, batchDuration: Duration) = this(sparkContext, null, batchDuration)
+
+ /**
+ * Creates a StreamingContext by providing the details necessary for creating a new SparkContext.
+ * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param frameworkName A name for your job, to display on the cluster web UI
+ * @param batchDuration The time interval at which streaming data will be divided into batches
+ */
+ def this(master: String, frameworkName: String, batchDuration: Duration) =
+ this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration)
+
+ /**
+ * Re-creates a StreamingContext from a checkpoint file.
+ * @param path Path either to the directory that was specified as the checkpoint directory, or
+ * to the checkpoint file 'graph' or 'graph.bk'.
+ */
+ def this(path: String) = this(null, CheckpointReader.read(path), null)
+
+ initLogging()
+
+ if (sc_ == null && cp_ == null) {
+ throw new Exception("Streaming Context cannot be initilalized with " +
+ "both SparkContext and checkpoint as null")
+ }
+
+ protected[streaming] val isCheckpointPresent = (cp_ != null)
+
+ val sc: SparkContext = {
+ if (isCheckpointPresent) {
+ new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars)
+ } else {
+ sc_
+ }
+ }
+
+ protected[streaming] val env = SparkEnv.get
+
+ protected[streaming] val graph: DStreamGraph = {
+ if (isCheckpointPresent) {
+ cp_.graph.setContext(this)
+ cp_.graph.restoreCheckpointData()
+ cp_.graph
+ } else {
+ assert(batchDur_ != null, "Batch duration for streaming context cannot be null")
+ val newGraph = new DStreamGraph()
+ newGraph.setBatchDuration(batchDur_)
+ newGraph
+ }
+ }
+
+ protected[streaming] val nextNetworkInputStreamId = new AtomicInteger(0)
+ protected[streaming] var networkInputTracker: NetworkInputTracker = null
+
+ protected[streaming] var checkpointDir: String = {
+ if (isCheckpointPresent) {
+ sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true)
+ cp_.checkpointDir
+ } else {
+ null
+ }
+ }
+
+ protected[streaming] var checkpointDuration: Duration = if (isCheckpointPresent) cp_.checkpointDuration else null
+ protected[streaming] var receiverJobThread: Thread = null
+ protected[streaming] var scheduler: Scheduler = null
+
+ /**
+ * Sets each DStreams in this context to remember RDDs it generated in the last given duration.
+ * DStreams remember RDDs only for a limited duration of time and releases them for garbage
+ * collection. This method allows the developer to specify how to long to remember the RDDs (
+ * if the developer wishes to query old data outside the DStream computation).
+ * @param duration Minimum duration that each DStream should remember its RDDs
+ */
+ def remember(duration: Duration) {
+ graph.remember(duration)
+ }
+
+ /**
+ * Sets the context to periodically checkpoint the DStream operations for master
+ * fault-tolerance. By default, the graph will be checkpointed every batch interval.
+ * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
+ * @param interval checkpoint interval
+ */
+ def checkpoint(directory: String, interval: Duration = null) {
+ if (directory != null) {
+ sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
+ checkpointDir = directory
+ checkpointDuration = interval
+ } else {
+ checkpointDir = null
+ checkpointDuration = null
+ }
+ }
+
+ protected[streaming] def getInitialCheckpoint(): Checkpoint = {
+ if (isCheckpointPresent) cp_ else null
+ }
+
+ protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement()
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param hostname Zookeper hostname.
+ * @param port Zookeper port.
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+ * By default the value is pulled from zookeper.
+ * @param storageLevel RDD storage level. Defaults to memory-only.
+ */
+ def kafkaStream[T: ClassManifest](
+ hostname: String,
+ port: Int,
+ groupId: String,
+ topics: Map[String, Int],
+ initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](),
+ storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
+ ): DStream[T] = {
+ val inputStream = new KafkaInputDStream[T](this, hostname, port, groupId, topics, initialOffsets, storageLevel)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create a input stream from network source hostname:port. Data is received using
+ * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
+ * lines.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param storageLevel Storage level to use for storing the received objects
+ * (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+ */
+ def networkTextStream(
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[String] = {
+ networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
+ }
+
+ /**
+ * Create a input stream from network source hostname:port. Data is received using
+ * a TCP socket and the receive bytes it interepreted as object using the given
+ * converter.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param converter Function to convert the byte stream to objects
+ * @param storageLevel Storage level to use for storing the received objects
+ * @tparam T Type of the objects received (after converting bytes to objects)
+ */
+ def networkStream[T: ClassManifest](
+ hostname: String,
+ port: Int,
+ converter: (InputStream) => Iterator[T],
+ storageLevel: StorageLevel
+ ): DStream[T] = {
+ val inputStream = new SocketInputDStream[T](this, hostname, port, converter, storageLevel)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Creates a input stream from a Flume source.
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def flumeStream (
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[SparkFlumeEvent] = {
+ val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create a input stream from network source hostname:port, where data is received
+ * as serialized blocks (serialized using the Spark's serializer) that can be directly
+ * pushed into the block manager without deserializing them. This is the most efficient
+ * way to receive data.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param storageLevel Storage level to use for storing the received objects
+ * @tparam T Type of the objects in the received blocks
+ */
+ def rawNetworkStream[T: ClassManifest](
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[T] = {
+ val inputStream = new RawInputDStream[T](this, hostname, port, storageLevel)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them using the given key-value types and input format.
+ * File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ * @tparam K Key type for reading HDFS file
+ * @tparam V Value type for reading HDFS file
+ * @tparam F Input format for reading HDFS file
+ */
+ def fileStream[
+ K: ClassManifest,
+ V: ClassManifest,
+ F <: NewInputFormat[K, V]: ClassManifest
+ ] (directory: String): DStream[(K, V)] = {
+ val inputStream = new FileInputDStream[K, V, F](this, directory)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them using the given key-value types and input format.
+ * @param directory HDFS directory to monitor for new file
+ * @param filter Function to filter paths to process
+ * @param newFilesOnly Should process only new files and ignore existing files in the directory
+ * @tparam K Key type for reading HDFS file
+ * @tparam V Value type for reading HDFS file
+ * @tparam F Input format for reading HDFS file
+ */
+ def fileStream[
+ K: ClassManifest,
+ V: ClassManifest,
+ F <: NewInputFormat[K, V]: ClassManifest
+ ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = {
+ val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+
+ /**
+ * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them as text files (using key as LongWritable, value
+ * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ */
+ def textFileStream(directory: String): DStream[String] = {
+ fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
+ }
+
+ /**
+ * Creates a input stream from an queue of RDDs. In each batch,
+ * it will process either one or all of the RDDs returned by the queue.
+ * @param queue Queue of RDDs
+ * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+ * @param defaultRDD Default RDD is returned by the DStream when the queue is empty
+ * @tparam T Type of objects in the RDD
+ */
+ def queueStream[T: ClassManifest](
+ queue: Queue[RDD[T]],
+ oneAtATime: Boolean = true,
+ defaultRDD: RDD[T] = null
+ ): DStream[T] = {
+ val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
+ registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create a unified DStream from multiple DStreams of the same type and same interval
+ */
+ def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
+ new UnionDStream[T](streams.toArray)
+ }
+
+ /**
+ * Registers an input stream that will be started (InputDStream.start() called) to get the
+ * input data.
+ */
+ def registerInputStream(inputStream: InputDStream[_]) {
+ graph.addInputStream(inputStream)
+ }
+
+ /**
+ * Registers an output stream that will be computed every interval
+ */
+ def registerOutputStream(outputStream: DStream[_]) {
+ graph.addOutputStream(outputStream)
+ }
+
+ protected def validate() {
+ assert(graph != null, "Graph is null")
+ graph.validate()
+
+ assert(
+ checkpointDir == null || checkpointDuration != null,
+ "Checkpoint directory has been set, but the graph checkpointing interval has " +
+ "not been set. Please use StreamingContext.checkpoint() to set the interval."
+ )
+ }
+
+ /**
+ * Starts the execution of the streams.
+ */
+ def start() {
+ if (checkpointDir != null && checkpointDuration == null && graph != null) {
+ checkpointDuration = graph.batchDuration
+ }
+
+ validate()
+
+ val networkInputStreams = graph.getInputStreams().filter(s => s match {
+ case n: NetworkInputDStream[_] => true
+ case _ => false
+ }).map(_.asInstanceOf[NetworkInputDStream[_]]).toArray
+
+ if (networkInputStreams.length > 0) {
+ // Start the network input tracker (must start before receivers)
+ networkInputTracker = new NetworkInputTracker(this, networkInputStreams)
+ networkInputTracker.start()
+ }
+
+ Thread.sleep(1000)
+
+ // Start the scheduler
+ scheduler = new Scheduler(this)
+ scheduler.start()
+ }
+
+ /**
+ * Sstops the execution of the streams.
+ */
+ def stop() {
+ try {
+ if (scheduler != null) scheduler.stop()
+ if (networkInputTracker != null) networkInputTracker.stop()
+ if (receiverJobThread != null) receiverJobThread.interrupt()
+ sc.stop()
+ logInfo("StreamingContext stopped successfully")
+ } catch {
+ case e: Exception => logWarning("Error while stopping", e)
+ }
+ }
+}
+
+
+object StreamingContext {
+
+ implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = {
+ new PairDStreamFunctions[K, V](stream)
+ }
+
+ protected[streaming] def createNewSparkContext(master: String, frameworkName: String): SparkContext = {
+
+ // Set the default cleaner delay to an hour if not already set.
+ // This should be sufficient for even 1 second interval.
+ if (MetadataCleaner.getDelaySeconds < 0) {
+ MetadataCleaner.setDelaySeconds(60)
+ }
+ new SparkContext(master, frameworkName)
+ }
+
+ protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
+ if (prefix == null) {
+ time.milliseconds.toString
+ } else if (suffix == null || suffix.length ==0) {
+ prefix + "-" + time.milliseconds
+ } else {
+ prefix + "-" + time.milliseconds + "." + suffix
+ }
+ }
+
+ protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = {
+ new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala
new file mode 100644
index 0000000000..5daeb761dd
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Time.scala
@@ -0,0 +1,42 @@
+package spark.streaming
+
+/**
+ * This is a simple class that represents an absolute instant of time.
+ * Internally, it represents time as the difference, measured in milliseconds, between the current
+ * time and midnight, January 1, 1970 UTC. This is the same format as what is returned by
+ * System.currentTimeMillis.
+ */
+case class Time(private val millis: Long) {
+
+ def milliseconds: Long = millis
+
+ def < (that: Time): Boolean = (this.millis < that.millis)
+
+ def <= (that: Time): Boolean = (this.millis <= that.millis)
+
+ def > (that: Time): Boolean = (this.millis > that.millis)
+
+ def >= (that: Time): Boolean = (this.millis >= that.millis)
+
+ def + (that: Duration): Time = new Time(millis + that.milliseconds)
+
+ def - (that: Time): Duration = new Duration(millis - that.millis)
+
+ def - (that: Duration): Time = new Time(millis - that.milliseconds)
+
+ def floor(that: Duration): Time = {
+ val t = that.milliseconds
+ val m = math.floor(this.millis / t).toLong
+ new Time(m * t)
+ }
+
+ def isMultipleOf(that: Duration): Boolean =
+ (this.millis % that.milliseconds == 0)
+
+ def min(that: Time): Time = if (this < that) this else that
+
+ def max(that: Time): Time = if (this > that) this else that
+
+ override def toString: String = (millis.toString + " ms")
+
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
new file mode 100644
index 0000000000..2e7466b16c
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
@@ -0,0 +1,91 @@
+package spark.streaming.api.java
+
+import spark.streaming.{Duration, Time, DStream}
+import spark.api.java.function.{Function => JFunction}
+import spark.api.java.JavaRDD
+import spark.storage.StorageLevel
+
+/**
+ * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * for more details on RDDs). DStreams can either be created from live data (such as, data from
+ * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
+ * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
+ * DStream periodically generates a RDD, either from live data or by transforming the RDD generated
+ * by a parent DStream.
+ *
+ * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
+ * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available
+ * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations
+ * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through
+ * implicit conversions when `spark.streaming.StreamingContext._` is imported.
+ *
+ * DStreams internally is characterized by a few basic properties:
+ * - A list of other DStreams that the DStream depends on
+ * - A time interval at which the DStream generates an RDD
+ * - A function that is used to generate an RDD after each time interval
+ */
+class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T])
+ extends JavaDStreamLike[T, JavaDStream[T]] {
+
+ /** Return a new DStream containing only the elements that satisfy a predicate. */
+ def filter(f: JFunction[T, java.lang.Boolean]): JavaDStream[T] =
+ dstream.filter((x => f(x).booleanValue()))
+
+ /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def cache(): JavaDStream[T] = dstream.cache()
+
+ /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def persist(): JavaDStream[T] = dstream.cache()
+
+ /** Persist the RDDs of this DStream with the given storage level */
+ def persist(storageLevel: StorageLevel): JavaDStream[T] = dstream.persist(storageLevel)
+
+ /** Generate an RDD for the given duration */
+ def compute(validTime: Time): JavaRDD[T] = {
+ dstream.compute(validTime) match {
+ case Some(rdd) => new JavaRDD(rdd)
+ case None => null
+ }
+ }
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of 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.
+ * @return
+ */
+ def window(windowDuration: Duration): JavaDStream[T] =
+ dstream.window(windowDuration)
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of this DStream.
+ * @param windowDuration duration (i.e., width) of the window;
+ * must be a multiple of this DStream's 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 interval
+ */
+ def window(windowDuration: Duration, slideDuration: Duration): JavaDStream[T] =
+ dstream.window(windowDuration, slideDuration)
+
+ /**
+ * Return a new DStream which computed based on tumbling window on this DStream.
+ * This is equivalent to window(batchDuration, batchDuration).
+ * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval
+ */
+ def tumble(batchDuration: Duration): JavaDStream[T] =
+ dstream.tumble(batchDuration)
+
+ /**
+ * Return a new DStream by unifying data of another DStream with this DStream.
+ * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream.
+ */
+ def union(that: JavaDStream[T]): JavaDStream[T] =
+ dstream.union(that.dstream)
+}
+
+object JavaDStream {
+ implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] =
+ new JavaDStream[T](dstream)
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
new file mode 100644
index 0000000000..b93cb7865a
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
@@ -0,0 +1,183 @@
+package spark.streaming.api.java
+
+import java.util.{List => JList}
+import java.lang.{Long => JLong}
+
+import scala.collection.JavaConversions._
+
+import spark.streaming._
+import spark.api.java.JavaRDD
+import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
+import java.util
+import spark.RDD
+import JavaDStream._
+
+trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable {
+ implicit val classManifest: ClassManifest[T]
+
+ def dstream: DStream[T]
+
+ implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = {
+ in.map(new JLong(_))
+ }
+
+ /**
+ * Print the first ten elements of each RDD generated in this DStream. This is an output
+ * operator, so this DStream will be registered as an output stream and there materialized.
+ */
+ def print() = dstream.print()
+
+ /**
+ * Return a new DStream in which each RDD has a single element generated by counting each RDD
+ * of this DStream.
+ */
+ def count(): JavaDStream[JLong] = dstream.count()
+
+ /**
+ * 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
+ * window() operation. This is equivalent to window(windowDuration, slideDuration).count()
+ */
+ def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[JLong] = {
+ dstream.countByWindow(windowDuration, slideDuration)
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying glom() to each RDD of
+ * this DStream. Applying glom() to an RDD coalesces all elements within each partition into
+ * an array.
+ */
+ def glom(): JavaDStream[JList[T]] =
+ new JavaDStream(dstream.glom().map(x => new java.util.ArrayList[T](x.toSeq)))
+
+ /** Return the StreamingContext associated with this DStream */
+ def context(): StreamingContext = dstream.context()
+
+ /** Return a new DStream by applying a function to all elements of this DStream. */
+ def map[R](f: JFunction[T, R]): JavaDStream[R] = {
+ new JavaDStream(dstream.map(f)(f.returnType()))(f.returnType())
+ }
+
+ /** Return a new DStream by applying a function to all elements of this DStream. */
+ def map[K, V](f: PairFunction[T, K, V]): JavaPairDStream[K, V] = {
+ def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType())
+ }
+
+ /**
+ * Return a new DStream by applying a function to all elements of this DStream,
+ * and then flattening the results
+ */
+ def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = {
+ import scala.collection.JavaConverters._
+ def fn = (x: T) => f.apply(x).asScala
+ new JavaDStream(dstream.flatMap(fn)(f.elementType()))(f.elementType())
+ }
+
+ /**
+ * Return a new DStream by applying a function to all elements of this DStream,
+ * and then flattening the results
+ */
+ def flatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairDStream[K, V] = {
+ import scala.collection.JavaConverters._
+ def fn = (x: T) => f.apply(x).asScala
+ def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType())
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+ * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+ * of the RDD.
+ */
+ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = {
+ def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
+ new JavaDStream(dstream.mapPartitions(fn)(f.elementType()))(f.elementType())
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+ * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+ * of the RDD.
+ */
+ def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V])
+ : JavaPairDStream[K, V] = {
+ def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
+ new JavaPairDStream(dstream.mapPartitions(fn))(f.keyType(), f.valueType())
+ }
+
+ /**
+ * Return a new DStream in which each RDD has a single element generated by reducing each RDD
+ * of this DStream.
+ */
+ def reduce(f: JFunction2[T, T, T]): JavaDStream[T] = dstream.reduce(f)
+
+ /**
+ * 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 in the
+ * window() operation. This is equivalent to window(windowDuration, slideDuration).reduce(reduceFunc)
+ */
+ def reduceByWindow(
+ reduceFunc: JFunction2[T, T, T],
+ invReduceFunc: JFunction2[T, T, T],
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): JavaDStream[T] = {
+ dstream.reduceByWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration)
+ }
+
+ /**
+ * Return all the RDDs between 'fromDuration' to 'toDuration' (both included)
+ */
+ def slice(fromDuration: Duration, toDuration: Duration): JList[JavaRDD[T]] = {
+ new util.ArrayList(dstream.slice(fromDuration, toDuration).map(new JavaRDD(_)).toSeq)
+ }
+
+ /**
+ * 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.
+ */
+ def foreach(foreachFunc: JFunction[JavaRDD[T], Void]) {
+ dstream.foreach(rdd => foreachFunc.call(new JavaRDD(rdd)))
+ }
+
+ /**
+ * 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.
+ */
+ def foreach(foreachFunc: JFunction2[JavaRDD[T], Time, Void]) {
+ dstream.foreach((rdd, time) => foreachFunc.call(new JavaRDD(rdd), time))
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying a function
+ * on each RDD of this DStream.
+ */
+ def transform[U](transformFunc: JFunction[JavaRDD[T], JavaRDD[U]]): JavaDStream[U] = {
+ implicit val cm: ClassManifest[U] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ def scalaTransform (in: RDD[T]): RDD[U] =
+ transformFunc.call(new JavaRDD[T](in)).rdd
+ dstream.transform(scalaTransform(_))
+ }
+
+ /**
+ * Return a new DStream in which each RDD is generated by applying a function
+ * on each RDD of this DStream.
+ */
+ def transform[U](transformFunc: JFunction2[JavaRDD[T], Time, JavaRDD[U]]): JavaDStream[U] = {
+ implicit val cm: ClassManifest[U] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ def scalaTransform (in: RDD[T], time: Time): RDD[U] =
+ transformFunc.call(new JavaRDD[T](in), time).rdd
+ dstream.transform(scalaTransform(_, _))
+ }
+
+ /**
+ * Enable periodic checkpointing of RDDs of this DStream
+ * @param interval Time interval after which generated RDD will be checkpointed
+ */
+ def checkpoint(interval: Duration) = {
+ dstream.checkpoint(interval)
+ }
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
new file mode 100644
index 0000000000..ef10c091ca
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
@@ -0,0 +1,638 @@
+package spark.streaming.api.java
+
+import java.util.{List => JList}
+import java.lang.{Long => JLong}
+
+import scala.collection.JavaConversions._
+
+import spark.streaming._
+import spark.streaming.StreamingContext._
+import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import spark.Partitioner
+import org.apache.hadoop.mapred.{JobConf, OutputFormat}
+import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
+import org.apache.hadoop.conf.Configuration
+import spark.api.java.JavaPairRDD
+import spark.storage.StorageLevel
+import com.google.common.base.Optional
+
+class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
+ implicit val kManifiest: ClassManifest[K],
+ implicit val vManifest: ClassManifest[V])
+ extends JavaDStreamLike[(K, V), JavaPairDStream[K, V]] {
+
+ // =======================================================================
+ // Methods common to all DStream's
+ // =======================================================================
+
+ /** Returns a new DStream containing only the elements that satisfy a predicate. */
+ def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairDStream[K, V] =
+ dstream.filter((x => f(x).booleanValue()))
+
+ /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def cache(): JavaPairDStream[K, V] = dstream.cache()
+
+ /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+ def persist(): JavaPairDStream[K, V] = dstream.cache()
+
+ /** Persists the RDDs of this DStream with the given storage level */
+ def persist(storageLevel: StorageLevel): JavaPairDStream[K, V] = dstream.persist(storageLevel)
+
+ /** Method that generates a RDD for the given Duration */
+ def compute(validTime: Time): JavaPairRDD[K, V] = {
+ dstream.compute(validTime) match {
+ case Some(rdd) => new JavaPairRDD(rdd)
+ case None => null
+ }
+ }
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of 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.
+ * @return
+ */
+ def window(windowDuration: Duration): JavaPairDStream[K, V] =
+ dstream.window(windowDuration)
+
+ /**
+ * Return a new DStream which is computed based on windowed batches of this DStream.
+ * @param windowDuration duration (i.e., width) of the window;
+ * must be a multiple of this DStream's 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 interval
+ */
+ def window(windowDuration: Duration, slideDuration: Duration): JavaPairDStream[K, V] =
+ dstream.window(windowDuration, slideDuration)
+
+ /**
+ * Returns a new DStream which computed based on tumbling window on this DStream.
+ * This is equivalent to window(batchDuration, batchDuration).
+ * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval
+ */
+ def tumble(batchDuration: Duration): JavaPairDStream[K, V] =
+ dstream.tumble(batchDuration)
+
+ /**
+ * Returns a new DStream by unifying data of another DStream with this DStream.
+ * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream.
+ */
+ def union(that: JavaPairDStream[K, V]): JavaPairDStream[K, V] =
+ dstream.union(that.dstream)
+
+ // =======================================================================
+ // Methods only for PairDStream's
+ // =======================================================================
+
+ /**
+ * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+ * generate the RDDs with Spark's default number of partitions.
+ */
+ def groupByKey(): JavaPairDStream[K, JList[V]] =
+ dstream.groupByKey().mapValues(seqAsJavaList _)
+
+ /**
+ * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+ * generate the RDDs with `numPartitions` partitions.
+ */
+ def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] =
+ dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _)
+
+ /**
+ * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
+ * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]]
+ * is used to control the partitioning of each RDD.
+ */
+ def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] =
+ dstream.groupByKey(partitioner).mapValues(seqAsJavaList _)
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the associative reduce function. Hash partitioning is used to generate the RDDs
+ * with Spark's default number of partitions.
+ */
+ def reduceByKey(func: JFunction2[V, V, V]): JavaPairDStream[K, V] =
+ dstream.reduceByKey(func)
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs
+ * with `numPartitions` partitions.
+ */
+ def reduceByKey(func: JFunction2[V, V, V], numPartitions: Int): JavaPairDStream[K, V] =
+ dstream.reduceByKey(func, numPartitions)
+
+ /**
+ * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+ * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+ * partitioning of each RDD.
+ */
+ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = {
+ dstream.reduceByKey(func, partitioner)
+ }
+
+ /**
+ * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+ * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+ * information.
+ */
+ def combineByKey[C](createCombiner: JFunction[V, C],
+ mergeValue: JFunction2[C, V, C],
+ mergeCombiners: JFunction2[C, C, C],
+ partitioner: Partitioner
+ ): JavaPairDStream[K, C] = {
+ implicit val cm: ClassManifest[C] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
+ }
+
+ /**
+ * Create a new DStream by counting the number of values of each key in each RDD. Hash
+ * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions.
+ */
+ def countByKey(numPartitions: Int): JavaPairDStream[K, JLong] = {
+ JavaPairDStream.scalaToJavaLong(dstream.countByKey(numPartitions));
+ }
+
+
+ /**
+ * Create a new DStream by counting the number of values of each key in each RDD. Hash
+ * partitioning is used to generate the RDDs with the default number of partitions.
+ */
+ def countByKey(): JavaPairDStream[K, JLong] = {
+ JavaPairDStream.scalaToJavaLong(dstream.countByKey());
+ }
+
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to
+ * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
+ * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
+ * Spark's default number of partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
+ def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = {
+ dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _)
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window. Similar to
+ * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with 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 groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration)
+ : JavaPairDStream[K, JList[V]] = {
+ dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _)
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
+ :JavaPairDStream[K, JList[V]] = {
+ dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions)
+ .mapValues(seqAsJavaList _)
+ }
+
+ /**
+ * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+ * @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 partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def groupByKeyAndWindow(
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ):JavaPairDStream[K, JList[V]] = {
+ dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner)
+ .mapValues(seqAsJavaList _)
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream
+ * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate
+ * the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
+ def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration)
+ :JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(reduceFunc, windowDuration)
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+ * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative 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 reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration
+ ):JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration)
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+ * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+ * generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int
+ ): JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions)
+ }
+
+ /**
+ * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
+ * `DStream.reduceByKey()`, but applies it over a sliding window.
+ * @param reduceFunc associative 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
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ): JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner)
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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 reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ invReduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration)
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ invReduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration,
+ numPartitions: Int
+ ): JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(
+ reduceFunc,
+ invReduceFunc,
+ windowDuration,
+ slideDuration,
+ numPartitions)
+ }
+
+ /**
+ * Create a new DStream by reducing over a using incremental computation.
+ * The reduced value of over a new window is calculated using the old window's reduce 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 that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse 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
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
+ def reduceByKeyAndWindow(
+ reduceFunc: Function2[V, V, V],
+ invReduceFunc: Function2[V, V, V],
+ windowDuration: Duration,
+ slideDuration: Duration,
+ partitioner: Partitioner
+ ): JavaPairDStream[K, V] = {
+ dstream.reduceByKeyAndWindow(
+ reduceFunc,
+ invReduceFunc,
+ windowDuration,
+ slideDuration,
+ partitioner)
+ }
+
+ /**
+ * Create a new DStream by counting the number of values for each key over a window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` 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 countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration)
+ : JavaPairDStream[K, JLong] = {
+ JavaPairDStream.scalaToJavaLong(dstream.countByKeyAndWindow(windowDuration, slideDuration))
+ }
+
+ /**
+ * Create a new DStream by counting the number of values for each key over a window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` 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
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
+ def countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
+ : JavaPairDStream[K, Long] = {
+ dstream.countByKeyAndWindow(windowDuration, slideDuration, numPartitions)
+ }
+
+ private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]):
+ (Seq[V], Option[S]) => Option[S] = {
+ val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => {
+ val list: JList[V] = values
+ val scalaState: Optional[S] = state match {
+ case Some(s) => Optional.of(s)
+ case _ => Optional.absent()
+ }
+ val result: Optional[S] = in.apply(list, scalaState)
+ result.isPresent match {
+ case true => Some(result.get())
+ case _ => None
+ }
+ }
+ scalaFunc
+ }
+
+ /**
+ * Create 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.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @tparam S State type
+ */
+ def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]])
+ : JavaPairDStream[K, S] = {
+ implicit val cm: ClassManifest[S] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+ dstream.updateStateByKey(convertUpdateStateFunction(updateFunc))
+ }
+
+ /**
+ * Create 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.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
+ numPartitions: Int)
+ : JavaPairDStream[K, S] = {
+ dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), numPartitions)
+ }
+
+ /**
+ * Create 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 the key.
+ * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ * @tparam S State type
+ */
+ def updateStateByKey[S: ClassManifest](
+ updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
+ partitioner: Partitioner
+ ): JavaPairDStream[K, S] = {
+ dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner)
+ }
+
+ def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = {
+ implicit val cm: ClassManifest[U] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ dstream.mapValues(f)
+ }
+
+ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
+ import scala.collection.JavaConverters._
+ def fn = (x: V) => f.apply(x).asScala
+ implicit val cm: ClassManifest[U] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ dstream.flatMapValues(fn)
+ }
+
+ /**
+ * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+ * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+ * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+ * of partitions.
+ */
+ def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = {
+ implicit val cm: ClassManifest[W] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+ }
+
+ /**
+ * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+ * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+ * key in both RDDs. Partitioner is used to partition each generated RDD.
+ */
+ def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
+ : JavaPairDStream[K, (JList[V], JList[W])] = {
+ implicit val cm: ClassManifest[W] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ dstream.cogroup(other.dstream, partitioner)
+ .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+ }
+
+ /**
+ * Join `this` DStream with `other` DStream. HashPartitioner is used
+ * to partition each generated RDD into default number of partitions.
+ */
+ def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = {
+ implicit val cm: ClassManifest[W] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ dstream.join(other.dstream)
+ }
+
+ /**
+ * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
+ * be generated by joining RDDs from `this` and other DStream. Uses the given
+ * Partitioner to partition each generated RDD.
+ */
+ def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
+ : JavaPairDStream[K, (V, W)] = {
+ implicit val cm: ClassManifest[W] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ dstream.join(other.dstream, partitioner)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsHadoopFiles[F <: OutputFormat[K, V]](prefix: String, suffix: String) {
+ dstream.saveAsHadoopFiles(prefix, suffix)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: OutputFormat[_, _]]) {
+ dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: OutputFormat[_, _]],
+ conf: JobConf) {
+ dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](prefix: String, suffix: String) {
+ dstream.saveAsNewAPIHadoopFiles(prefix, suffix)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsNewAPIHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
+ dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass)
+ }
+
+ /**
+ * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+ * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
+ def saveAsNewAPIHadoopFiles(
+ prefix: String,
+ suffix: String,
+ keyClass: Class[_],
+ valueClass: Class[_],
+ outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
+ conf: Configuration = new Configuration) {
+ dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
+ }
+
+ override val classManifest: ClassManifest[(K, V)] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+}
+
+object JavaPairDStream {
+ implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)])
+ :JavaPairDStream[K, V] =
+ new JavaPairDStream[K, V](dstream)
+
+ def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
+ implicit val cmk: ClassManifest[K] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+ implicit val cmv: ClassManifest[V] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ new JavaPairDStream[K, V](dstream.dstream)
+ }
+
+ def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long])
+ : JavaPairDStream[K, JLong] = {
+ StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
new file mode 100644
index 0000000000..f82e6a37cc
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -0,0 +1,346 @@
+package spark.streaming.api.java
+
+import scala.collection.JavaConversions._
+import java.lang.{Long => JLong, Integer => JInt}
+
+import spark.streaming._
+import dstream._
+import spark.storage.StorageLevel
+import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import java.io.InputStream
+import java.util.{Map => JMap}
+import spark.api.java.{JavaSparkContext, JavaRDD}
+
+/**
+ * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
+ * information (such as, cluster URL and job name) to internally create a SparkContext, it provides
+ * methods used to create DStream from various input sources.
+ */
+class JavaStreamingContext(val ssc: StreamingContext) {
+
+ // TODOs:
+ // - Test to/from Hadoop functions
+ // - Support creating and registering InputStreams
+
+
+ /**
+ * Creates a StreamingContext.
+ * @param master Name of the Spark Master
+ * @param frameworkName Name to be used when registering with the scheduler
+ * @param batchDuration The time interval at which streaming data will be divided into batches
+ */
+ def this(master: String, frameworkName: String, batchDuration: Duration) =
+ this(new StreamingContext(master, frameworkName, batchDuration))
+
+ /**
+ * Re-creates a StreamingContext from a checkpoint file.
+ * @param path Path either to the directory that was specified as the checkpoint directory, or
+ * to the checkpoint file 'graph' or 'graph.bk'.
+ */
+ def this(path: String) = this (new StreamingContext(path))
+
+ /** The underlying SparkContext */
+ val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param hostname Zookeper hostname.
+ * @param port Zookeper port.
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ */
+ def kafkaStream[T](
+ hostname: String,
+ port: Int,
+ groupId: String,
+ topics: JMap[String, JInt])
+ : JavaDStream[T] = {
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ ssc.kafkaStream[T](hostname, port, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+ }
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param hostname Zookeper hostname.
+ * @param port Zookeper port.
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+ * By default the value is pulled from zookeper.
+ */
+ def kafkaStream[T](
+ hostname: String,
+ port: Int,
+ groupId: String,
+ topics: JMap[String, JInt],
+ initialOffsets: JMap[KafkaPartitionKey, JLong])
+ : JavaDStream[T] = {
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ ssc.kafkaStream[T](
+ hostname,
+ port,
+ groupId,
+ Map(topics.mapValues(_.intValue()).toSeq: _*),
+ Map(initialOffsets.mapValues(_.longValue()).toSeq: _*))
+ }
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param hostname Zookeper hostname.
+ * @param port Zookeper port.
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+ * By default the value is pulled from zookeper.
+ * @param storageLevel RDD storage level. Defaults to memory-only
+ */
+ def kafkaStream[T](
+ hostname: String,
+ port: Int,
+ groupId: String,
+ topics: JMap[String, JInt],
+ initialOffsets: JMap[KafkaPartitionKey, JLong],
+ storageLevel: StorageLevel)
+ : JavaDStream[T] = {
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ ssc.kafkaStream[T](
+ hostname,
+ port,
+ groupId,
+ Map(topics.mapValues(_.intValue()).toSeq: _*),
+ Map(initialOffsets.mapValues(_.longValue()).toSeq: _*),
+ storageLevel)
+ }
+
+ /**
+ * Create a input stream from network source hostname:port. Data is received using
+ * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
+ * lines.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param storageLevel Storage level to use for storing the received objects
+ * (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+ */
+ def networkTextStream(hostname: String, port: Int, storageLevel: StorageLevel)
+ : JavaDStream[String] = {
+ ssc.networkTextStream(hostname, port, storageLevel)
+ }
+
+ /**
+ * Create a input stream from network source hostname:port. Data is received using
+ * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
+ * lines.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ */
+ def networkTextStream(hostname: String, port: Int): JavaDStream[String] = {
+ ssc.networkTextStream(hostname, port)
+ }
+
+ /**
+ * Create a input stream from network source hostname:port. Data is received using
+ * a TCP socket and the receive bytes it interepreted as object using the given
+ * converter.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param converter Function to convert the byte stream to objects
+ * @param storageLevel Storage level to use for storing the received objects
+ * @tparam T Type of the objects received (after converting bytes to objects)
+ */
+ def networkStream[T](
+ hostname: String,
+ port: Int,
+ converter: JFunction[InputStream, java.lang.Iterable[T]],
+ storageLevel: StorageLevel)
+ : JavaDStream[T] = {
+ def fn = (x: InputStream) => converter.apply(x).toIterator
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ ssc.networkStream(hostname, port, fn, storageLevel)
+ }
+
+ /**
+ * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them as text files (using key as LongWritable, value
+ * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ */
+ def textFileStream(directory: String): JavaDStream[String] = {
+ ssc.textFileStream(directory)
+ }
+
+ /**
+ * Create a input stream from network source hostname:port, where data is received
+ * as serialized blocks (serialized using the Spark's serializer) that can be directly
+ * pushed into the block manager without deserializing them. This is the most efficient
+ * way to receive data.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @param storageLevel Storage level to use for storing the received objects
+ * @tparam T Type of the objects in the received blocks
+ */
+ def rawNetworkStream[T](
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel): JavaDStream[T] = {
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port, storageLevel))
+ }
+
+ /**
+ * Create a input stream from network source hostname:port, where data is received
+ * as serialized blocks (serialized using the Spark's serializer) that can be directly
+ * pushed into the block manager without deserializing them. This is the most efficient
+ * way to receive data.
+ * @param hostname Hostname to connect to for receiving data
+ * @param port Port to connect to for receiving data
+ * @tparam T Type of the objects in the received blocks
+ */
+ def rawNetworkStream[T](hostname: String, port: Int): JavaDStream[T] = {
+ implicit val cmt: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port))
+ }
+
+ /**
+ * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * for new files and reads them using the given key-value types and input format.
+ * File names starting with . are ignored.
+ * @param directory HDFS directory to monitor for new file
+ * @tparam K Key type for reading HDFS file
+ * @tparam V Value type for reading HDFS file
+ * @tparam F Input format for reading HDFS file
+ */
+ def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = {
+ implicit val cmk: ClassManifest[K] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+ implicit val cmv: ClassManifest[V] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cmf: ClassManifest[F] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]]
+ ssc.fileStream[K, V, F](directory);
+ }
+
+ /**
+ * Creates a input stream from a Flume source.
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
+ JavaDStream[SparkFlumeEvent] = {
+ ssc.flumeStream(hostname, port, storageLevel)
+ }
+
+
+ /**
+ * Creates a input stream from a Flume source.
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ */
+ def flumeStream(hostname: String, port: Int):
+ JavaDStream[SparkFlumeEvent] = {
+ ssc.flumeStream(hostname, port)
+ }
+
+ /**
+ * Registers an output stream that will be computed every interval
+ */
+ def registerOutputStream(outputStream: JavaDStreamLike[_, _]) {
+ ssc.registerOutputStream(outputStream.dstream)
+ }
+
+ /**
+ * Creates a input stream from an queue of RDDs. In each batch,
+ * it will process either one or all of the RDDs returned by the queue.
+ *
+ * NOTE: changes to the queue after the stream is created will not be recognized.
+ * @param queue Queue of RDDs
+ * @tparam T Type of objects in the RDD
+ */
+ def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = {
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+ sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+ ssc.queueStream(sQueue)
+ }
+
+ /**
+ * Creates a input stream from an queue of RDDs. In each batch,
+ * it will process either one or all of the RDDs returned by the queue.
+ *
+ * NOTE: changes to the queue after the stream is created will not be recognized.
+ * @param queue Queue of RDDs
+ * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+ * @tparam T Type of objects in the RDD
+ */
+ def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = {
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+ sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+ ssc.queueStream(sQueue, oneAtATime)
+ }
+
+ /**
+ * Creates a input stream from an queue of RDDs. In each batch,
+ * it will process either one or all of the RDDs returned by the queue.
+ *
+ * NOTE: changes to the queue after the stream is created will not be recognized.
+ * @param queue Queue of RDDs
+ * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+ * @param defaultRDD Default RDD is returned by the DStream when the queue is empty
+ * @tparam T Type of objects in the RDD
+ */
+ def queueStream[T](
+ queue: java.util.Queue[JavaRDD[T]],
+ oneAtATime: Boolean,
+ defaultRDD: JavaRDD[T]): JavaDStream[T] = {
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+ sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+ ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
+ }
+
+ /**
+ * Sets the context to periodically checkpoint the DStream operations for master
+ * fault-tolerance. By default, the graph will be checkpointed every batch interval.
+ * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
+ * @param interval checkpoint interval
+ */
+ def checkpoint(directory: String, interval: Duration = null) {
+ ssc.checkpoint(directory, interval)
+ }
+
+ /**
+ * Sets each DStreams in this context to remember RDDs it generated in the last given duration.
+ * DStreams remember RDDs only for a limited duration of duration and releases them for garbage
+ * collection. This method allows the developer to specify how to long to remember the RDDs (
+ * if the developer wishes to query old data outside the DStream computation).
+ * @param duration Minimum duration that each DStream should remember its RDDs
+ */
+ def remember(duration: Duration) {
+ ssc.remember(duration)
+ }
+
+ /**
+ * Starts the execution of the streams.
+ */
+ def start() = ssc.start()
+
+ /**
+ * Sstops the execution of the streams.
+ */
+ def stop() = ssc.stop()
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
new file mode 100644
index 0000000000..ddb1bf6b28
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.{RDD, Partitioner}
+import spark.rdd.CoGroupedRDD
+import spark.streaming.{Time, DStream, Duration}
+
+private[streaming]
+class CoGroupedDStream[K : ClassManifest](
+ parents: Seq[DStream[(_, _)]],
+ partitioner: Partitioner
+ ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) {
+
+ if (parents.length == 0) {
+ throw new IllegalArgumentException("Empty array of parents")
+ }
+
+ if (parents.map(_.ssc).distinct.size > 1) {
+ throw new IllegalArgumentException("Array of parents have different StreamingContexts")
+ }
+
+ if (parents.map(_.slideDuration).distinct.size > 1) {
+ throw new IllegalArgumentException("Array of parents have different slide times")
+ }
+
+ override def dependencies = parents.toList
+
+ override def slideDuration: Duration = parents.head.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = {
+ val part = partitioner
+ val rdds = parents.flatMap(_.getOrCompute(validTime))
+ if (rdds.size > 0) {
+ val q = new CoGroupedRDD[K](rdds, part)
+ Some(q)
+ } else {
+ None
+ }
+ }
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
new file mode 100644
index 0000000000..41c3af4694
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
@@ -0,0 +1,19 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Time, StreamingContext}
+
+/**
+ * An input stream that always returns the same RDD on each timestep. Useful for testing.
+ */
+class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T])
+ extends InputDStream[T](ssc_) {
+
+ override def start() {}
+
+ override def stop() {}
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ Some(rdd)
+ }
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
new file mode 100644
index 0000000000..1e6ad84b44
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
@@ -0,0 +1,102 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+import spark.streaming.{StreamingContext, Time}
+
+import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+
+import scala.collection.mutable.HashSet
+
+private[streaming]
+class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
+ @transient ssc_ : StreamingContext,
+ directory: String,
+ filter: Path => Boolean = FileInputDStream.defaultFilter,
+ newFilesOnly: Boolean = true)
+ extends InputDStream[(K, V)](ssc_) {
+
+ @transient private var path_ : Path = null
+ @transient private var fs_ : FileSystem = null
+
+ var lastModTime = 0L
+ val lastModTimeFiles = new HashSet[String]()
+
+ def path(): Path = {
+ if (path_ == null) path_ = new Path(directory)
+ path_
+ }
+
+ def fs(): FileSystem = {
+ if (fs_ == null) fs_ = path.getFileSystem(new Configuration())
+ fs_
+ }
+
+ override def start() {
+ if (newFilesOnly) {
+ lastModTime = System.currentTimeMillis()
+ } else {
+ lastModTime = 0
+ }
+ }
+
+ override def stop() { }
+
+ /**
+ * Finds the files that were modified since the last time this method was called and makes
+ * a union RDD out of them. Note that this maintains the list of files that were processed
+ * in the latest modification time in the previous call to this method. This is because the
+ * modification time returned by the FileStatus API seems to return times only at the
+ * granularity of seconds. Hence, new files may have the same modification time as the
+ * latest modification time in the previous call to this method and the list of files
+ * maintained is used to filter the one that have been processed.
+ */
+ override def compute(validTime: Time): Option[RDD[(K, V)]] = {
+ // Create the filter for selecting new files
+ val newFilter = new PathFilter() {
+ var latestModTime = 0L
+ val latestModTimeFiles = new HashSet[String]()
+
+ def accept(path: Path): Boolean = {
+ if (!filter(path)) {
+ return false
+ } else {
+ val modTime = fs.getFileStatus(path).getModificationTime()
+ if (modTime < lastModTime){
+ return false
+ } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) {
+ return false
+ }
+ if (modTime > latestModTime) {
+ latestModTime = modTime
+ latestModTimeFiles.clear()
+ }
+ latestModTimeFiles += path.toString
+ return true
+ }
+ }
+ }
+
+ val newFiles = fs.listStatus(path, newFilter)
+ logInfo("New files: " + newFiles.map(_.getPath).mkString(", "))
+ if (newFiles.length > 0) {
+ // Update the modification time and the files processed for that modification time
+ if (lastModTime != newFilter.latestModTime) {
+ lastModTime = newFilter.latestModTime
+ lastModTimeFiles.clear()
+ }
+ lastModTimeFiles ++= newFilter.latestModTimeFiles
+ }
+ val newRDD = new UnionRDD(ssc.sc, newFiles.map(
+ file => ssc.sc.newAPIHadoopFile[K, V, F](file.getPath.toString)))
+ Some(newRDD)
+ }
+}
+
+private[streaming]
+object FileInputDStream {
+ def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
new file mode 100644
index 0000000000..e993164f99
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class FilteredDStream[T: ClassManifest](
+ parent: DStream[T],
+ filterFunc: T => Boolean
+ ) extends DStream[T](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ parent.getOrCompute(validTime).map(_.filter(filterFunc))
+ }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
new file mode 100644
index 0000000000..cabd34f5f2
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import spark.SparkContext._
+
+private[streaming]
+class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+ parent: DStream[(K, V)],
+ flatMapValueFunc: V => TraversableOnce[U]
+ ) extends DStream[(K, U)](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[(K, U)]] = {
+ parent.getOrCompute(validTime).map(_.flatMapValues[U](flatMapValueFunc))
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
new file mode 100644
index 0000000000..a69af60589
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
+ parent: DStream[T],
+ flatMapFunc: T => Traversable[U]
+ ) extends DStream[U](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[U]] = {
+ parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc))
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
new file mode 100644
index 0000000000..efc7058480
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
@@ -0,0 +1,137 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext
+
+import spark.Utils
+import spark.storage.StorageLevel
+
+import org.apache.flume.source.avro.AvroSourceProtocol
+import org.apache.flume.source.avro.AvroFlumeEvent
+import org.apache.flume.source.avro.Status
+import org.apache.avro.ipc.specific.SpecificResponder
+import org.apache.avro.ipc.NettyServer
+
+import scala.collection.JavaConversions._
+
+import java.net.InetSocketAddress
+import java.io.{ObjectInput, ObjectOutput, Externalizable}
+import java.nio.ByteBuffer
+
+private[streaming]
+class FlumeInputDStream[T: ClassManifest](
+ @transient ssc_ : StreamingContext,
+ host: String,
+ port: Int,
+ storageLevel: StorageLevel
+) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
+
+ override def createReceiver(): NetworkReceiver[SparkFlumeEvent] = {
+ new FlumeReceiver(host, port, storageLevel)
+ }
+}
+
+/**
+ * A wrapper class for AvroFlumeEvent's with a custom serialization format.
+ *
+ * This is necessary because AvroFlumeEvent uses inner data structures
+ * which are not serializable.
+ */
+class SparkFlumeEvent() extends Externalizable {
+ var event : AvroFlumeEvent = new AvroFlumeEvent()
+
+ /* De-serialize from bytes. */
+ def readExternal(in: ObjectInput) {
+ val bodyLength = in.readInt()
+ val bodyBuff = new Array[Byte](bodyLength)
+ in.read(bodyBuff)
+
+ val numHeaders = in.readInt()
+ val headers = new java.util.HashMap[CharSequence, CharSequence]
+
+ for (i <- 0 until numHeaders) {
+ val keyLength = in.readInt()
+ val keyBuff = new Array[Byte](keyLength)
+ in.read(keyBuff)
+ val key : String = Utils.deserialize(keyBuff)
+
+ val valLength = in.readInt()
+ val valBuff = new Array[Byte](valLength)
+ in.read(valBuff)
+ val value : String = Utils.deserialize(valBuff)
+
+ headers.put(key, value)
+ }
+
+ event.setBody(ByteBuffer.wrap(bodyBuff))
+ event.setHeaders(headers)
+ }
+
+ /* Serialize to bytes. */
+ def writeExternal(out: ObjectOutput) {
+ val body = event.getBody.array()
+ out.writeInt(body.length)
+ out.write(body)
+
+ val numHeaders = event.getHeaders.size()
+ out.writeInt(numHeaders)
+ for ((k, v) <- event.getHeaders) {
+ val keyBuff = Utils.serialize(k.toString)
+ out.writeInt(keyBuff.length)
+ out.write(keyBuff)
+ val valBuff = Utils.serialize(v.toString)
+ out.writeInt(valBuff.length)
+ out.write(valBuff)
+ }
+ }
+}
+
+private[streaming] object SparkFlumeEvent {
+ def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = {
+ val event = new SparkFlumeEvent
+ event.event = in
+ event
+ }
+}
+
+/** A simple server that implements Flume's Avro protocol. */
+private[streaming]
+class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol {
+ override def append(event : AvroFlumeEvent) : Status = {
+ receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)
+ Status.OK
+ }
+
+ override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = {
+ events.foreach (event =>
+ receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event))
+ Status.OK
+ }
+}
+
+/** A NetworkReceiver which listens for events using the
+ * Flume Avro interface.*/
+private[streaming]
+class FlumeReceiver(
+ host: String,
+ port: Int,
+ storageLevel: StorageLevel
+ ) extends NetworkReceiver[SparkFlumeEvent] {
+
+ lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+ protected override def onStart() {
+ val responder = new SpecificResponder(
+ classOf[AvroSourceProtocol], new FlumeEventServer(this));
+ val server = new NettyServer(responder, new InetSocketAddress(host, port));
+ blockGenerator.start()
+ server.start()
+ logInfo("Flume receiver started")
+ }
+
+ protected override def onStop() {
+ blockGenerator.stop()
+ logInfo("Flume receiver stopped")
+ }
+
+ override def getLocationPreference = Some(host)
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
new file mode 100644
index 0000000000..ee69ea5177
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
@@ -0,0 +1,28 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Duration, DStream, Job, Time}
+
+private[streaming]
+class ForEachDStream[T: ClassManifest] (
+ parent: DStream[T],
+ foreachFunc: (RDD[T], Time) => Unit
+ ) extends DStream[Unit](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[Unit]] = None
+
+ override def generateJob(time: Time): Option[Job] = {
+ parent.getOrCompute(time) match {
+ case Some(rdd) =>
+ val jobFunc = () => {
+ foreachFunc(rdd, time)
+ }
+ Some(new Job(time, jobFunc))
+ case None => None
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
new file mode 100644
index 0000000000..b589cbd4d5
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
@@ -0,0 +1,17 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class GlommedDStream[T: ClassManifest](parent: DStream[T])
+ extends DStream[Array[T]](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[Array[T]]] = {
+ parent.getOrCompute(validTime).map(_.glom())
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
new file mode 100644
index 0000000000..980ca5177e
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
@@ -0,0 +1,19 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, StreamingContext, DStream}
+
+abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
+ extends DStream[T](ssc_) {
+
+ override def dependencies = List()
+
+ override def slideDuration: Duration = {
+ if (ssc == null) throw new Exception("ssc is null")
+ if (ssc.graph.batchDuration == null) throw new Exception("batchDuration is null")
+ ssc.graph.batchDuration
+ }
+
+ def start()
+
+ def stop()
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
new file mode 100644
index 0000000000..2b4740bdf7
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
@@ -0,0 +1,200 @@
+package spark.streaming.dstream
+
+import spark.Logging
+import spark.storage.StorageLevel
+import spark.streaming.{Time, DStreamCheckpointData, StreamingContext}
+
+import java.util.Properties
+import java.util.concurrent.Executors
+
+import kafka.consumer._
+import kafka.message.{Message, MessageSet, MessageAndMetadata}
+import kafka.serializer.StringDecoder
+import kafka.utils.{Utils, ZKGroupTopicDirs}
+import kafka.utils.ZkUtils._
+
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+
+
+// Key for a specific Kafka Partition: (broker, topic, group, part)
+case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int)
+// NOT USED - Originally intended for fault-tolerance
+// Metadata for a Kafka Stream that it sent to the Master
+private[streaming]
+case class KafkaInputDStreamMetadata(timestamp: Long, data: Map[KafkaPartitionKey, Long])
+// NOT USED - Originally intended for fault-tolerance
+// Checkpoint data specific to a KafkaInputDstream
+private[streaming]
+case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any],
+ savedOffsets: Map[KafkaPartitionKey, Long]) extends DStreamCheckpointData(kafkaRdds)
+
+/**
+ * Input stream that pulls messages from a Kafka Broker.
+ *
+ * @param host Zookeper hostname.
+ * @param port Zookeper port.
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+ * By default the value is pulled from zookeper.
+ * @param storageLevel RDD storage level.
+ */
+private[streaming]
+class KafkaInputDStream[T: ClassManifest](
+ @transient ssc_ : StreamingContext,
+ host: String,
+ port: Int,
+ groupId: String,
+ topics: Map[String, Int],
+ initialOffsets: Map[KafkaPartitionKey, Long],
+ storageLevel: StorageLevel
+ ) extends NetworkInputDStream[T](ssc_ ) with Logging {
+
+ // Metadata that keeps track of which messages have already been consumed.
+ var savedOffsets = HashMap[Long, Map[KafkaPartitionKey, Long]]()
+
+ /* NOT USED - Originally intended for fault-tolerance
+
+ // In case of a failure, the offets for a particular timestamp will be restored.
+ @transient var restoredOffsets : Map[KafkaPartitionKey, Long] = null
+
+
+ override protected[streaming] def addMetadata(metadata: Any) {
+ metadata match {
+ case x : KafkaInputDStreamMetadata =>
+ savedOffsets(x.timestamp) = x.data
+ // TOOD: Remove logging
+ logInfo("New saved Offsets: " + savedOffsets)
+ case _ => logInfo("Received unknown metadata: " + metadata.toString)
+ }
+ }
+
+ override protected[streaming] def updateCheckpointData(currentTime: Time) {
+ super.updateCheckpointData(currentTime)
+ if(savedOffsets.size > 0) {
+ // Find the offets that were stored before the checkpoint was initiated
+ val key = savedOffsets.keys.toList.sortWith(_ < _).filter(_ < currentTime.millis).last
+ val latestOffsets = savedOffsets(key)
+ logInfo("Updating KafkaDStream checkpoint data: " + latestOffsets.toString)
+ checkpointData = KafkaDStreamCheckpointData(checkpointData.rdds, latestOffsets)
+ // TODO: This may throw out offsets that are created after the checkpoint,
+ // but it's unlikely we'll need them.
+ savedOffsets.clear()
+ }
+ }
+
+ override protected[streaming] def restoreCheckpointData() {
+ super.restoreCheckpointData()
+ logInfo("Restoring KafkaDStream checkpoint data.")
+ checkpointData match {
+ case x : KafkaDStreamCheckpointData =>
+ restoredOffsets = x.savedOffsets
+ logInfo("Restored KafkaDStream offsets: " + savedOffsets)
+ }
+ } */
+
+ def createReceiver(): NetworkReceiver[T] = {
+ new KafkaReceiver(host, port, groupId, topics, initialOffsets, storageLevel)
+ .asInstanceOf[NetworkReceiver[T]]
+ }
+}
+
+private[streaming]
+class KafkaReceiver(host: String, port: Int, groupId: String,
+ topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long],
+ storageLevel: StorageLevel) extends NetworkReceiver[Any] {
+
+ // Timeout for establishing a connection to Zookeper in ms.
+ val ZK_TIMEOUT = 10000
+
+ // Handles pushing data into the BlockManager
+ lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+ // Keeps track of the current offsets. Maps from (broker, topic, group, part) -> Offset
+ lazy val offsets = HashMap[KafkaPartitionKey, Long]()
+ // Connection to Kafka
+ var consumerConnector : ZookeeperConsumerConnector = null
+
+ def onStop() {
+ blockGenerator.stop()
+ }
+
+ def onStart() {
+
+ blockGenerator.start()
+
+ // In case we are using multiple Threads to handle Kafka Messages
+ val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
+
+ val zooKeeperEndPoint = host + ":" + port
+ logInfo("Starting Kafka Consumer Stream with group: " + groupId)
+ logInfo("Initial offsets: " + initialOffsets.toString)
+
+ // Zookeper connection properties
+ val props = new Properties()
+ props.put("zk.connect", zooKeeperEndPoint)
+ props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString)
+ props.put("groupid", groupId)
+
+ // Create the connection to the cluster
+ logInfo("Connecting to Zookeper: " + zooKeeperEndPoint)
+ val consumerConfig = new ConsumerConfig(props)
+ consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector]
+ logInfo("Connected to " + zooKeeperEndPoint)
+
+ // Reset the Kafka offsets in case we are recovering from a failure
+ resetOffsets(initialOffsets)
+
+ // Create Threads for each Topic/Message Stream we are listening
+ val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder())
+
+ // Start the messages handler for each partition
+ topicMessageStreams.values.foreach { streams =>
+ streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
+ }
+
+ }
+
+ // Overwrites the offets in Zookeper.
+ private def resetOffsets(offsets: Map[KafkaPartitionKey, Long]) {
+ offsets.foreach { case(key, offset) =>
+ val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic)
+ val partitionName = key.brokerId + "-" + key.partId
+ updatePersistentPath(consumerConnector.zkClient,
+ topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString)
+ }
+ }
+
+ // Handles Kafka Messages
+ private class MessageHandler(stream: KafkaStream[String]) extends Runnable {
+ def run() {
+ logInfo("Starting MessageHandler.")
+ stream.takeWhile { msgAndMetadata =>
+ blockGenerator += msgAndMetadata.message
+
+ // Updating the offet. The key is (broker, topic, group, partition).
+ val key = KafkaPartitionKey(msgAndMetadata.topicInfo.brokerId, msgAndMetadata.topic,
+ groupId, msgAndMetadata.topicInfo.partition.partId)
+ val offset = msgAndMetadata.topicInfo.getConsumeOffset
+ offsets.put(key, offset)
+ // logInfo("Handled message: " + (key, offset).toString)
+
+ // Keep on handling messages
+ true
+ }
+ }
+ }
+
+ // NOT USED - Originally intended for fault-tolerance
+ // class KafkaDataHandler(receiver: KafkaReceiver, storageLevel: StorageLevel)
+ // extends BufferingBlockCreator[Any](receiver, storageLevel) {
+
+ // override def createBlock(blockId: String, iterator: Iterator[Any]) : Block = {
+ // // Creates a new Block with Kafka-specific Metadata
+ // new Block(blockId, iterator, KafkaInputDStreamMetadata(System.currentTimeMillis, offsets.toMap))
+ // }
+
+ // }
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
new file mode 100644
index 0000000000..848afecfad
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
+ parent: DStream[T],
+ mapPartFunc: Iterator[T] => Iterator[U],
+ preservePartitioning: Boolean
+ ) extends DStream[U](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[U]] = {
+ parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning))
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
new file mode 100644
index 0000000000..6055aa6a05
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import spark.SparkContext._
+
+private[streaming]
+class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+ parent: DStream[(K, V)],
+ mapValueFunc: V => U
+ ) extends DStream[(K, U)](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[(K, U)]] = {
+ parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc))
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
new file mode 100644
index 0000000000..20818a0cab
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class MappedDStream[T: ClassManifest, U: ClassManifest] (
+ parent: DStream[T],
+ mapFunc: T => U
+ ) extends DStream[U](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[U]] = {
+ parent.getOrCompute(validTime).map(_.map[U](mapFunc))
+ }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
new file mode 100644
index 0000000000..aa6be95f30
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
@@ -0,0 +1,254 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver}
+
+import spark.{Logging, SparkEnv, RDD}
+import spark.rdd.BlockRDD
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+
+import java.nio.ByteBuffer
+
+import akka.actor.{Props, Actor}
+import akka.pattern.ask
+import akka.dispatch.Await
+import akka.util.duration._
+import spark.streaming.util.{RecurringTimer, SystemClock}
+import java.util.concurrent.ArrayBlockingQueue
+
+/**
+ * Abstract class for defining any InputDStream that has to start a receiver on worker
+ * nodes to receive external data. Specific implementations of NetworkInputDStream must
+ * define the createReceiver() function that creates the receiver object of type
+ * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive
+ * data.
+ * @param ssc_ Streaming context that will execute this input stream
+ * @tparam T Class type of the object of this stream
+ */
+abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : StreamingContext)
+ extends InputDStream[T](ssc_) {
+
+ // This is an unique identifier that is used to match the network receiver with the
+ // corresponding network input stream.
+ val id = ssc.getNewNetworkStreamId()
+
+ /**
+ * Creates the receiver object that will be sent to the worker nodes
+ * to receive data. This method needs to defined by any specific implementation
+ * of a NetworkInputDStream.
+ */
+ def createReceiver(): NetworkReceiver[T]
+
+ // Nothing to start or stop as both taken care of by the NetworkInputTracker.
+ def start() {}
+
+ def stop() {}
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
+ Some(new BlockRDD[T](ssc.sc, blockIds))
+ }
+}
+
+
+private[streaming] sealed trait NetworkReceiverMessage
+private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage
+private[streaming] case class ReportBlock(blockId: String, metadata: Any) extends NetworkReceiverMessage
+private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage
+
+/**
+ * Abstract class of a receiver that can be run on worker nodes to receive external data. See
+ * [[spark.streaming.dstream.NetworkInputDStream]] for an explanation.
+ */
+abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging {
+
+ initLogging()
+
+ lazy protected val env = SparkEnv.get
+
+ lazy protected val actor = env.actorSystem.actorOf(
+ Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId)
+
+ lazy protected val receivingThread = Thread.currentThread()
+
+ protected var streamId: Int = -1
+
+ /**
+ * This method will be called to start receiving data. All your receiver
+ * starting code should be implemented by defining this function.
+ */
+ protected def onStart()
+
+ /** This method will be called to stop receiving data. */
+ protected def onStop()
+
+ /** Conveys a placement preference (hostname) for this receiver. */
+ def getLocationPreference() : Option[String] = None
+
+ /**
+ * Starts the receiver. First is accesses all the lazy members to
+ * materialize them. Then it calls the user-defined onStart() method to start
+ * other threads, etc required to receiver the data.
+ */
+ def start() {
+ try {
+ // Access the lazy vals to materialize them
+ env
+ actor
+ receivingThread
+
+ // Call user-defined onStart()
+ onStart()
+ } catch {
+ case ie: InterruptedException =>
+ logInfo("Receiving thread interrupted")
+ //println("Receiving thread interrupted")
+ case e: Exception =>
+ stopOnError(e)
+ }
+ }
+
+ /**
+ * Stops the receiver. First it interrupts the main receiving thread,
+ * that is, the thread that called receiver.start(). Then it calls the user-defined
+ * onStop() method to stop other threads and/or do cleanup.
+ */
+ def stop() {
+ receivingThread.interrupt()
+ onStop()
+ //TODO: terminate the actor
+ }
+
+ /**
+ * Stops the receiver and reports to exception to the tracker.
+ * This should be called whenever an exception has happened on any thread
+ * of the receiver.
+ */
+ protected def stopOnError(e: Exception) {
+ logError("Error receiving data", e)
+ stop()
+ actor ! ReportError(e.toString)
+ }
+
+
+ /**
+ * Pushes a block (as iterator of values) into the block manager.
+ */
+ def pushBlock(blockId: String, iterator: Iterator[T], metadata: Any, level: StorageLevel) {
+ val buffer = new ArrayBuffer[T] ++ iterator
+ env.blockManager.put(blockId, buffer.asInstanceOf[ArrayBuffer[Any]], level)
+
+ actor ! ReportBlock(blockId, metadata)
+ }
+
+ /**
+ * Pushes a block (as bytes) into the block manager.
+ */
+ def pushBlock(blockId: String, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
+ env.blockManager.putBytes(blockId, bytes, level)
+ actor ! ReportBlock(blockId, metadata)
+ }
+
+ /** A helper actor that communicates with the NetworkInputTracker */
+ private class NetworkReceiverActor extends Actor {
+ logInfo("Attempting to register with tracker")
+ val ip = System.getProperty("spark.master.host", "localhost")
+ val port = System.getProperty("spark.master.port", "7077").toInt
+ val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
+ val tracker = env.actorSystem.actorFor(url)
+ val timeout = 5.seconds
+
+ override def preStart() {
+ val future = tracker.ask(RegisterReceiver(streamId, self))(timeout)
+ Await.result(future, timeout)
+ }
+
+ override def receive() = {
+ case ReportBlock(blockId, metadata) =>
+ tracker ! AddBlocks(streamId, Array(blockId), metadata)
+ case ReportError(msg) =>
+ tracker ! DeregisterReceiver(streamId, msg)
+ case StopReceiver(msg) =>
+ stop()
+ tracker ! DeregisterReceiver(streamId, msg)
+ }
+ }
+
+ protected[streaming] def setStreamId(id: Int) {
+ streamId = id
+ }
+
+ /**
+ * Batches objects created by a [[spark.streaming.NetworkReceiver]] and puts them into
+ * appropriately named blocks at regular intervals. This class starts two threads,
+ * one to periodically start a new batch and prepare the previous batch of as a block,
+ * the other to push the blocks into the block manager.
+ */
+ class BlockGenerator(storageLevel: StorageLevel)
+ extends Serializable with Logging {
+
+ case class Block(id: String, iterator: Iterator[T], metadata: Any = null)
+
+ val clock = new SystemClock()
+ val blockInterval = 200L
+ val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
+ val blockStorageLevel = storageLevel
+ val blocksForPushing = new ArrayBlockingQueue[Block](1000)
+ val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } }
+
+ var currentBuffer = new ArrayBuffer[T]
+
+ def start() {
+ blockIntervalTimer.start()
+ blockPushingThread.start()
+ logInfo("Data handler started")
+ }
+
+ def stop() {
+ blockIntervalTimer.stop()
+ blockPushingThread.interrupt()
+ logInfo("Data handler stopped")
+ }
+
+ def += (obj: T) {
+ currentBuffer += obj
+ }
+
+ private def createBlock(blockId: String, iterator: Iterator[T]) : Block = {
+ new Block(blockId, iterator)
+ }
+
+ private def updateCurrentBuffer(time: Long) {
+ try {
+ val newBlockBuffer = currentBuffer
+ currentBuffer = new ArrayBuffer[T]
+ if (newBlockBuffer.size > 0) {
+ val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval)
+ val newBlock = createBlock(blockId, newBlockBuffer.toIterator)
+ blocksForPushing.add(newBlock)
+ }
+ } catch {
+ case ie: InterruptedException =>
+ logInfo("Block interval timer thread interrupted")
+ case e: Exception =>
+ NetworkReceiver.this.stop()
+ }
+ }
+
+ private def keepPushingBlocks() {
+ logInfo("Block pushing thread started")
+ try {
+ while(true) {
+ val block = blocksForPushing.take()
+ NetworkReceiver.this.pushBlock(block.id, block.iterator, block.metadata, storageLevel)
+ }
+ } catch {
+ case ie: InterruptedException =>
+ logInfo("Block pushing thread interrupted")
+ case e: Exception =>
+ NetworkReceiver.this.stop()
+ }
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
new file mode 100644
index 0000000000..024bf3bea4
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
@@ -0,0 +1,41 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+
+import scala.collection.mutable.Queue
+import scala.collection.mutable.ArrayBuffer
+import spark.streaming.{Time, StreamingContext}
+
+class QueueInputDStream[T: ClassManifest](
+ @transient ssc: StreamingContext,
+ val queue: Queue[RDD[T]],
+ oneAtATime: Boolean,
+ defaultRDD: RDD[T]
+ ) extends InputDStream[T](ssc) {
+
+ override def start() { }
+
+ override def stop() { }
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ val buffer = new ArrayBuffer[RDD[T]]()
+ if (oneAtATime && queue.size > 0) {
+ buffer += queue.dequeue()
+ } else {
+ buffer ++= queue
+ }
+ if (buffer.size > 0) {
+ if (oneAtATime) {
+ Some(buffer.first)
+ } else {
+ Some(new UnionRDD(ssc.sc, buffer.toSeq))
+ }
+ } else if (defaultRDD != null) {
+ Some(defaultRDD)
+ } else {
+ None
+ }
+ }
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
new file mode 100644
index 0000000000..290fab1ce0
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
@@ -0,0 +1,90 @@
+package spark.streaming.dstream
+
+import spark.{DaemonThread, Logging}
+import spark.storage.StorageLevel
+import spark.streaming.StreamingContext
+
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+import java.nio.channels.{ReadableByteChannel, SocketChannel}
+import java.io.EOFException
+import java.util.concurrent.ArrayBlockingQueue
+
+
+/**
+ * An input stream that reads blocks of serialized objects from a given network address.
+ * The blocks will be inserted directly into the block store. This is the fastest way to get
+ * data into Spark Streaming, though it requires the sender to batch data and serialize it
+ * in the format that the system is configured with.
+ */
+private[streaming]
+class RawInputDStream[T: ClassManifest](
+ @transient ssc_ : StreamingContext,
+ host: String,
+ port: Int,
+ storageLevel: StorageLevel
+ ) extends NetworkInputDStream[T](ssc_ ) with Logging {
+
+ def createReceiver(): NetworkReceiver[T] = {
+ new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]]
+ }
+}
+
+private[streaming]
+class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel)
+ extends NetworkReceiver[Any] {
+
+ var blockPushingThread: Thread = null
+
+ override def getLocationPreference = None
+
+ def onStart() {
+ // Open a socket to the target address and keep reading from it
+ logInfo("Connecting to " + host + ":" + port)
+ val channel = SocketChannel.open()
+ channel.configureBlocking(true)
+ channel.connect(new InetSocketAddress(host, port))
+ logInfo("Connected to " + host + ":" + port)
+
+ val queue = new ArrayBlockingQueue[ByteBuffer](2)
+
+ blockPushingThread = new DaemonThread {
+ override def run() {
+ var nextBlockNumber = 0
+ while (true) {
+ val buffer = queue.take()
+ val blockId = "input-" + streamId + "-" + nextBlockNumber
+ nextBlockNumber += 1
+ pushBlock(blockId, buffer, null, storageLevel)
+ }
+ }
+ }
+ blockPushingThread.start()
+
+ val lengthBuffer = ByteBuffer.allocate(4)
+ while (true) {
+ lengthBuffer.clear()
+ readFully(channel, lengthBuffer)
+ lengthBuffer.flip()
+ val length = lengthBuffer.getInt()
+ val dataBuffer = ByteBuffer.allocate(length)
+ readFully(channel, dataBuffer)
+ dataBuffer.flip()
+ logInfo("Read a block with " + length + " bytes")
+ queue.put(dataBuffer)
+ }
+ }
+
+ def onStop() {
+ if (blockPushingThread != null) blockPushingThread.interrupt()
+ }
+
+ /** Read a buffer fully from a given Channel */
+ private def readFully(channel: ReadableByteChannel, dest: ByteBuffer) {
+ while (dest.position < dest.limit) {
+ if (channel.read(dest) == -1) {
+ throw new EOFException("End of channel")
+ }
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
new file mode 100644
index 0000000000..733d5c4a25
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -0,0 +1,149 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext._
+
+import spark.RDD
+import spark.rdd.CoGroupedRDD
+import spark.Partitioner
+import spark.SparkContext._
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+import spark.streaming.{Duration, Interval, Time, DStream}
+
+private[streaming]
+class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
+ parent: DStream[(K, V)],
+ reduceFunc: (V, V) => V,
+ invReduceFunc: (V, V) => V,
+ _windowDuration: Duration,
+ _slideDuration: Duration,
+ partitioner: Partitioner
+ ) extends DStream[(K,V)](parent.ssc) {
+
+ assert(_windowDuration.isMultipleOf(parent.slideDuration),
+ "The window duration of ReducedWindowedDStream (" + _slideDuration + ") " +
+ "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
+ )
+
+ assert(_slideDuration.isMultipleOf(parent.slideDuration),
+ "The slide duration of ReducedWindowedDStream (" + _slideDuration + ") " +
+ "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
+ )
+
+ // Reduce each batch of data using reduceByKey which will be further reduced by window
+ // by ReducedWindowedDStream
+ val reducedStream = parent.reduceByKey(reduceFunc, partitioner)
+
+ // Persist RDDs to memory by default as these RDDs are going to be reused.
+ super.persist(StorageLevel.MEMORY_ONLY_SER)
+ reducedStream.persist(StorageLevel.MEMORY_ONLY_SER)
+
+ def windowDuration: Duration = _windowDuration
+
+ override def dependencies = List(reducedStream)
+
+ override def slideDuration: Duration = _slideDuration
+
+ override val mustCheckpoint = true
+
+ override def parentRememberDuration: Duration = rememberDuration + windowDuration
+
+ override def persist(storageLevel: StorageLevel): DStream[(K,V)] = {
+ super.persist(storageLevel)
+ reducedStream.persist(storageLevel)
+ this
+ }
+
+ override def checkpoint(interval: Duration): DStream[(K, V)] = {
+ super.checkpoint(interval)
+ //reducedStream.checkpoint(interval)
+ this
+ }
+
+ override def compute(validTime: Time): Option[RDD[(K, V)]] = {
+ val reduceF = reduceFunc
+ val invReduceF = invReduceFunc
+
+ val currentTime = validTime
+ val currentWindow = new Interval(currentTime - windowDuration + parent.slideDuration, currentTime)
+ val previousWindow = currentWindow - slideDuration
+
+ logDebug("Window time = " + windowDuration)
+ logDebug("Slide time = " + slideDuration)
+ logDebug("ZeroTime = " + zeroTime)
+ logDebug("Current window = " + currentWindow)
+ logDebug("Previous window = " + previousWindow)
+
+ // _____________________________
+ // | previous window _________|___________________
+ // |___________________| current window | --------------> Time
+ // |_____________________________|
+ //
+ // |________ _________| |________ _________|
+ // | |
+ // V V
+ // old RDDs new RDDs
+ //
+
+ // Get the RDDs of the reduced values in "old time steps"
+ val oldRDDs = reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration)
+ logDebug("# old RDDs = " + oldRDDs.size)
+
+ // Get the RDDs of the reduced values in "new time steps"
+ val newRDDs = reducedStream.slice(previousWindow.endTime + parent.slideDuration, currentWindow.endTime)
+ logDebug("# new RDDs = " + newRDDs.size)
+
+ // Get the RDD of the reduced value of the previous window
+ val previousWindowRDD = getOrCompute(previousWindow.endTime).getOrElse(ssc.sc.makeRDD(Seq[(K,V)]()))
+
+ // Make the list of RDDs that needs to cogrouped together for reducing their reduced values
+ val allRDDs = new ArrayBuffer[RDD[(K, V)]]() += previousWindowRDD ++= oldRDDs ++= newRDDs
+
+ // Cogroup the reduced RDDs and merge the reduced values
+ val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(_, _)]]], partitioner)
+ //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _
+
+ val numOldValues = oldRDDs.size
+ val numNewValues = newRDDs.size
+
+ val mergeValues = (seqOfValues: Seq[Seq[V]]) => {
+ if (seqOfValues.size != 1 + numOldValues + numNewValues) {
+ throw new Exception("Unexpected number of sequences of reduced values")
+ }
+ // Getting reduced values "old time steps" that will be removed from current window
+ val oldValues = (1 to numOldValues).map(i => seqOfValues(i)).filter(!_.isEmpty).map(_.head)
+ // Getting reduced values "new time steps"
+ val newValues = (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head)
+ if (seqOfValues(0).isEmpty) {
+ // If previous window's reduce value does not exist, then at least new values should exist
+ if (newValues.isEmpty) {
+ throw new Exception("Neither previous window has value for key, nor new values found. " +
+ "Are you sure your key class hashes consistently?")
+ }
+ // Reduce the new values
+ newValues.reduce(reduceF) // return
+ } else {
+ // Get the previous window's reduced value
+ var tempValue = seqOfValues(0).head
+ // If old values exists, then inverse reduce then from previous value
+ if (!oldValues.isEmpty) {
+ tempValue = invReduceF(tempValue, oldValues.reduce(reduceF))
+ }
+ // If new values exists, then reduce them with previous value
+ if (!newValues.isEmpty) {
+ tempValue = reduceF(tempValue, newValues.reduce(reduceF))
+ }
+ tempValue // return
+ }
+ }
+
+ val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues)
+
+ Some(mergedValuesRDD)
+ }
+
+
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
new file mode 100644
index 0000000000..1f9548bfb8
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
@@ -0,0 +1,27 @@
+package spark.streaming.dstream
+
+import spark.{RDD, Partitioner}
+import spark.SparkContext._
+import spark.streaming.{Duration, DStream, Time}
+
+private[streaming]
+class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
+ parent: DStream[(K,V)],
+ createCombiner: V => C,
+ mergeValue: (C, V) => C,
+ mergeCombiner: (C, C) => C,
+ partitioner: Partitioner
+ ) extends DStream [(K,C)] (parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[(K,C)]] = {
+ parent.getOrCompute(validTime) match {
+ case Some(rdd) =>
+ Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
+ case None => None
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
new file mode 100644
index 0000000000..d42027092b
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
@@ -0,0 +1,103 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext
+import spark.storage.StorageLevel
+
+import java.io._
+import java.net.Socket
+
+private[streaming]
+class SocketInputDStream[T: ClassManifest](
+ @transient ssc_ : StreamingContext,
+ host: String,
+ port: Int,
+ bytesToObjects: InputStream => Iterator[T],
+ storageLevel: StorageLevel
+ ) extends NetworkInputDStream[T](ssc_) {
+
+ def createReceiver(): NetworkReceiver[T] = {
+ new SocketReceiver(host, port, bytesToObjects, storageLevel)
+ }
+}
+
+private[streaming]
+class SocketReceiver[T: ClassManifest](
+ host: String,
+ port: Int,
+ bytesToObjects: InputStream => Iterator[T],
+ storageLevel: StorageLevel
+ ) extends NetworkReceiver[T] {
+
+ lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+
+ override def getLocationPreference = None
+
+ protected def onStart() {
+ logInfo("Connecting to " + host + ":" + port)
+ val socket = new Socket(host, port)
+ logInfo("Connected to " + host + ":" + port)
+ blockGenerator.start()
+ val iterator = bytesToObjects(socket.getInputStream())
+ while(iterator.hasNext) {
+ val obj = iterator.next
+ blockGenerator += obj
+ }
+ }
+
+ protected def onStop() {
+ blockGenerator.stop()
+ }
+
+}
+
+private[streaming]
+object SocketReceiver {
+
+ /**
+ * This methods translates the data from an inputstream (say, from a socket)
+ * to '\n' delimited strings and returns an iterator to access the strings.
+ */
+ def bytesToLines(inputStream: InputStream): Iterator[String] = {
+ val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8"))
+
+ val iterator = new Iterator[String] {
+ var gotNext = false
+ var finished = false
+ var nextValue: String = null
+
+ private def getNext() {
+ try {
+ nextValue = dataInputStream.readLine()
+ if (nextValue == null) {
+ finished = true
+ }
+ }
+ gotNext = true
+ }
+
+ override def hasNext: Boolean = {
+ if (!finished) {
+ if (!gotNext) {
+ getNext()
+ if (finished) {
+ dataInputStream.close()
+ }
+ }
+ }
+ !finished
+ }
+
+ override def next(): String = {
+ if (finished) {
+ throw new NoSuchElementException("End of stream")
+ }
+ if (!gotNext) {
+ getNext()
+ }
+ gotNext = false
+ nextValue
+ }
+ }
+ iterator
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
new file mode 100644
index 0000000000..b4506c74aa
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
@@ -0,0 +1,84 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.Partitioner
+import spark.SparkContext._
+import spark.storage.StorageLevel
+import spark.streaming.{Duration, Time, DStream}
+
+private[streaming]
+class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
+ parent: DStream[(K, V)],
+ updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
+ partitioner: Partitioner,
+ preservePartitioning: Boolean
+ ) extends DStream[(K, S)](parent.ssc) {
+
+ super.persist(StorageLevel.MEMORY_ONLY_SER)
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override val mustCheckpoint = true
+
+ override def compute(validTime: Time): Option[RDD[(K, S)]] = {
+
+ // Try to get the previous state RDD
+ getOrCompute(validTime - slideDuration) match {
+
+ case Some(prevStateRDD) => { // If previous state RDD exists
+
+ // Try to get the parent RDD
+ parent.getOrCompute(validTime) match {
+ case Some(parentRDD) => { // If parent RDD exists, then compute as usual
+
+ // Define the function for the mapPartition operation on cogrouped RDD;
+ // first map the cogrouped tuple to tuples of required type,
+ // and then apply the update function
+ val updateFuncLocal = updateFunc
+ val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => {
+ val i = iterator.map(t => {
+ (t._1, t._2._1, t._2._2.headOption)
+ })
+ updateFuncLocal(i)
+ }
+ val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)
+ val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)
+ //logDebug("Generating state RDD for time " + validTime)
+ return Some(stateRDD)
+ }
+ case None => { // If parent RDD does not exist, then return old state RDD
+ return Some(prevStateRDD)
+ }
+ }
+ }
+
+ case None => { // If previous session RDD does not exist (first input data)
+
+ // Try to get the parent RDD
+ parent.getOrCompute(validTime) match {
+ case Some(parentRDD) => { // If parent RDD exists, then compute as usual
+
+ // Define the function for the mapPartition operation on grouped RDD;
+ // first map the grouped tuple to tuples of required type,
+ // and then apply the update function
+ val updateFuncLocal = updateFunc
+ val finalFunc = (iterator: Iterator[(K, Seq[V])]) => {
+ updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None)))
+ }
+
+ val groupedRDD = parentRDD.groupByKey(partitioner)
+ val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning)
+ //logDebug("Generating state RDD for time " + validTime + " (first)")
+ return Some(sessionRDD)
+ }
+ case None => { // If parent RDD does not exist, then nothing to do!
+ //logDebug("Not generating state RDD (no previous state, no parent)")
+ return None
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
new file mode 100644
index 0000000000..99660d9dee
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
@@ -0,0 +1,19 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Duration, DStream, Time}
+
+private[streaming]
+class TransformedDStream[T: ClassManifest, U: ClassManifest] (
+ parent: DStream[T],
+ transformFunc: (RDD[T], Time) => RDD[U]
+ ) extends DStream[U](parent.ssc) {
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = parent.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[U]] = {
+ parent.getOrCompute(validTime).map(transformFunc(_, validTime))
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
new file mode 100644
index 0000000000..00bad5da34
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import collection.mutable.ArrayBuffer
+import spark.rdd.UnionRDD
+
+private[streaming]
+class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
+ extends DStream[T](parents.head.ssc) {
+
+ if (parents.length == 0) {
+ throw new IllegalArgumentException("Empty array of parents")
+ }
+
+ if (parents.map(_.ssc).distinct.size > 1) {
+ throw new IllegalArgumentException("Array of parents have different StreamingContexts")
+ }
+
+ if (parents.map(_.slideDuration).distinct.size > 1) {
+ throw new IllegalArgumentException("Array of parents have different slide times")
+ }
+
+ override def dependencies = parents.toList
+
+ override def slideDuration: Duration = parents.head.slideDuration
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ val rdds = new ArrayBuffer[RDD[T]]()
+ parents.map(_.getOrCompute(validTime)).foreach(_ match {
+ case Some(rdd) => rdds += rdd
+ case None => throw new Exception("Could not generate RDD from a parent for unifying at time " + validTime)
+ })
+ if (rdds.size > 0) {
+ Some(new UnionRDD(ssc.sc, rdds))
+ } else {
+ None
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
new file mode 100644
index 0000000000..cbf0c88108
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+import spark.storage.StorageLevel
+import spark.streaming.{Duration, Interval, Time, DStream}
+
+private[streaming]
+class WindowedDStream[T: ClassManifest](
+ parent: DStream[T],
+ _windowDuration: Duration,
+ _slideDuration: Duration)
+ extends DStream[T](parent.ssc) {
+
+ if (!_windowDuration.isMultipleOf(parent.slideDuration))
+ throw new Exception("The window duration of WindowedDStream (" + _slideDuration + ") " +
+ "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")")
+
+ if (!_slideDuration.isMultipleOf(parent.slideDuration))
+ throw new Exception("The slide duration of WindowedDStream (" + _slideDuration + ") " +
+ "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")")
+
+ parent.persist(StorageLevel.MEMORY_ONLY_SER)
+
+ def windowDuration: Duration = _windowDuration
+
+ override def dependencies = List(parent)
+
+ override def slideDuration: Duration = _slideDuration
+
+ override def parentRememberDuration: Duration = rememberDuration + windowDuration
+
+ override def compute(validTime: Time): Option[RDD[T]] = {
+ val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
+ Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
+ }
+}
+
+
+
diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala
new file mode 100644
index 0000000000..974651f9f6
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala
@@ -0,0 +1,84 @@
+package spark.streaming.util
+
+private[streaming]
+trait Clock {
+ def currentTime(): Long
+ def waitTillTime(targetTime: Long): Long
+}
+
+private[streaming]
+class SystemClock() extends Clock {
+
+ val minPollTime = 25L
+
+ def currentTime(): Long = {
+ System.currentTimeMillis()
+ }
+
+ def waitTillTime(targetTime: Long): Long = {
+ var currentTime = 0L
+ currentTime = System.currentTimeMillis()
+
+ var waitTime = targetTime - currentTime
+ if (waitTime <= 0) {
+ return currentTime
+ }
+
+ val pollTime = {
+ if (waitTime / 10.0 > minPollTime) {
+ (waitTime / 10.0).toLong
+ } else {
+ minPollTime
+ }
+ }
+
+
+ while (true) {
+ currentTime = System.currentTimeMillis()
+ waitTime = targetTime - currentTime
+
+ if (waitTime <= 0) {
+
+ return currentTime
+ }
+ val sleepTime =
+ if (waitTime < pollTime) {
+ waitTime
+ } else {
+ pollTime
+ }
+ Thread.sleep(sleepTime)
+ }
+ return -1
+ }
+}
+
+private[streaming]
+class ManualClock() extends Clock {
+
+ var time = 0L
+
+ def currentTime() = time
+
+ def setTime(timeToSet: Long) = {
+ this.synchronized {
+ time = timeToSet
+ this.notifyAll()
+ }
+ }
+
+ def addToTime(timeToAdd: Long) = {
+ this.synchronized {
+ time += timeToAdd
+ this.notifyAll()
+ }
+ }
+ def waitTillTime(targetTime: Long): Long = {
+ this.synchronized {
+ while (time < targetTime) {
+ this.wait(100)
+ }
+ }
+ return currentTime()
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
new file mode 100644
index 0000000000..03749d4a94
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
@@ -0,0 +1,98 @@
+package spark.streaming.util
+
+import spark.SparkContext
+import spark.SparkContext._
+import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
+import scala.collection.JavaConversions.mapAsScalaMap
+
+object RawTextHelper {
+
+ /**
+ * Splits lines and counts the words in them using specialized object-to-long hashmap
+ * (to avoid boxing-unboxing overhead of Long in java/scala HashMap)
+ */
+ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = {
+ val map = new OLMap[String]
+ var i = 0
+ var j = 0
+ while (iter.hasNext) {
+ val s = iter.next()
+ i = 0
+ while (i < s.length) {
+ j = i
+ while (j < s.length && s.charAt(j) != ' ') {
+ j += 1
+ }
+ if (j > i) {
+ val w = s.substring(i, j)
+ val c = map.getLong(w)
+ map.put(w, c + 1)
+ }
+ i = j
+ while (i < s.length && s.charAt(i) == ' ') {
+ i += 1
+ }
+ }
+ }
+ map.toIterator.map{case (k, v) => (k, v)}
+ }
+
+ /**
+ * Gets the top k words in terms of word counts. Assumes that each word exists only once
+ * in the `data` iterator (that is, the counts have been reduced).
+ */
+ def topK(data: Iterator[(String, Long)], k: Int): Iterator[(String, Long)] = {
+ val taken = new Array[(String, Long)](k)
+
+ var i = 0
+ var len = 0
+ var done = false
+ var value: (String, Long) = null
+ var swap: (String, Long) = null
+ var count = 0
+
+ while(data.hasNext) {
+ value = data.next
+ if (value != null) {
+ count += 1
+ if (len == 0) {
+ taken(0) = value
+ len = 1
+ } else if (len < k || value._2 > taken(len - 1)._2) {
+ if (len < k) {
+ len += 1
+ }
+ taken(len - 1) = value
+ i = len - 1
+ while(i > 0 && taken(i - 1)._2 < taken(i)._2) {
+ swap = taken(i)
+ taken(i) = taken(i-1)
+ taken(i - 1) = swap
+ i -= 1
+ }
+ }
+ }
+ }
+ return taken.toIterator
+ }
+
+ /**
+ * Warms up the SparkContext in master and slave by running tasks to force JIT kick in
+ * before real workload starts.
+ */
+ def warmUp(sc: SparkContext) {
+ for(i <- 0 to 1) {
+ sc.parallelize(1 to 200000, 1000)
+ .map(_ % 1331).map(_.toString)
+ .mapPartitions(splitAndCountPartitions).reduceByKey(_ + _, 10)
+ .count()
+ }
+ }
+
+ def add(v1: Long, v2: Long) = (v1 + v2)
+
+ def subtract(v1: Long, v2: Long) = (v1 - v2)
+
+ def max(v1: Long, v2: Long) = math.max(v1, v2)
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
new file mode 100644
index 0000000000..d8b987ec86
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
@@ -0,0 +1,60 @@
+package spark.streaming.util
+
+import java.nio.ByteBuffer
+import spark.util.{RateLimitedOutputStream, IntParam}
+import java.net.ServerSocket
+import spark.{Logging, KryoSerializer}
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+import io.Source
+import java.io.IOException
+
+/**
+ * A helper program that sends blocks of Kryo-serialized text strings out on a socket at a
+ * specified rate. Used to feed data into RawInputDStream.
+ */
+object RawTextSender extends Logging {
+ def main(args: Array[String]) {
+ if (args.length != 4) {
+ System.err.println("Usage: RawTextSender <port> <file> <blockSize> <bytesPerSec>")
+ System.exit(1)
+ }
+ // Parse the arguments using a pattern match
+ val Array(IntParam(port), file, IntParam(blockSize), IntParam(bytesPerSec)) = args
+
+ // Repeat the input data multiple times to fill in a buffer
+ val lines = Source.fromFile(file).getLines().toArray
+ val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
+ val ser = new KryoSerializer().newInstance()
+ val serStream = ser.serializeStream(bufferStream)
+ var i = 0
+ while (bufferStream.position < blockSize) {
+ serStream.writeObject(lines(i))
+ i = (i + 1) % lines.length
+ }
+ bufferStream.trim()
+ val array = bufferStream.array
+
+ val countBuf = ByteBuffer.wrap(new Array[Byte](4))
+ countBuf.putInt(array.length)
+ countBuf.flip()
+
+ val serverSocket = new ServerSocket(port)
+ logInfo("Listening on port " + port)
+
+ while (true) {
+ val socket = serverSocket.accept()
+ logInfo("Got a new connection")
+ val out = new RateLimitedOutputStream(socket.getOutputStream, bytesPerSec)
+ try {
+ while (true) {
+ out.write(countBuf.array)
+ out.write(array)
+ }
+ } catch {
+ case e: IOException =>
+ logError("Client disconnected")
+ socket.close()
+ }
+ }
+ }
+}
diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
new file mode 100644
index 0000000000..db715cc295
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
@@ -0,0 +1,75 @@
+package spark.streaming.util
+
+private[streaming]
+class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) {
+
+ val minPollTime = 25L
+
+ val pollTime = {
+ if (period / 10.0 > minPollTime) {
+ (period / 10.0).toLong
+ } else {
+ minPollTime
+ }
+ }
+
+ val thread = new Thread() {
+ override def run() { loop }
+ }
+
+ var nextTime = 0L
+
+ def start(startTime: Long): Long = {
+ nextTime = startTime
+ thread.start()
+ nextTime
+ }
+
+ def start(): Long = {
+ val startTime = (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period
+ start(startTime)
+ }
+
+ def restart(originalStartTime: Long): Long = {
+ val gap = clock.currentTime - originalStartTime
+ val newStartTime = (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime
+ start(newStartTime)
+ }
+
+ def stop() {
+ thread.interrupt()
+ }
+
+ def loop() {
+ try {
+ while (true) {
+ clock.waitTillTime(nextTime)
+ callback(nextTime)
+ nextTime += period
+ }
+
+ } catch {
+ case e: InterruptedException =>
+ }
+ }
+}
+
+private[streaming]
+object RecurringTimer {
+
+ def main(args: Array[String]) {
+ var lastRecurTime = 0L
+ val period = 1000
+
+ def onRecur(time: Long) {
+ val currentTime = System.currentTimeMillis()
+ println("" + currentTime + ": " + (currentTime - lastRecurTime))
+ lastRecurTime = currentTime
+ }
+ val timer = new RecurringTimer(new SystemClock(), period, onRecur)
+ timer.start()
+ Thread.sleep(30 * 1000)
+ timer.stop()
+ }
+}
+
diff --git a/streaming/src/test/java/JavaAPISuite.java b/streaming/src/test/java/JavaAPISuite.java
new file mode 100644
index 0000000000..c84e7331c7
--- /dev/null
+++ b/streaming/src/test/java/JavaAPISuite.java
@@ -0,0 +1,1029 @@
+package spark.streaming;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Tuple2;
+import spark.HashPartitioner;
+import spark.api.java.JavaRDD;
+import spark.api.java.JavaSparkContext;
+import spark.api.java.function.*;
+import spark.storage.StorageLevel;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+import spark.streaming.JavaTestUtils;
+import spark.streaming.JavaCheckpointTestUtils;
+import spark.streaming.dstream.KafkaPartitionKey;
+
+import java.io.*;
+import java.util.*;
+
+// The test suite itself is Serializable so that anonymous Function implementations can be
+// serialized, as an alternative to converting these anonymous classes to static inner classes;
+// see http://stackoverflow.com/questions/758570/.
+public class JavaAPISuite implements Serializable {
+ private transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint", new Duration(1000));
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port");
+ }
+
+ @Test
+ public void testCount() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3,4),
+ Arrays.asList(3,4,5),
+ Arrays.asList(3));
+
+ List<List<Long>> expected = Arrays.asList(
+ Arrays.asList(4L),
+ Arrays.asList(3L),
+ Arrays.asList(1L));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream count = stream.count();
+ JavaTestUtils.attachTestOutputStream(count);
+ List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testMap() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("hello", "world"),
+ Arrays.asList("goodnight", "moon"));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(5,5),
+ Arrays.asList(9,4));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ @Override
+ public Integer call(String s) throws Exception {
+ return s.length();
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(letterCount);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testWindow() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6,1,2,3),
+ Arrays.asList(7,8,9,4,5,6),
+ Arrays.asList(7,8,9));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream windowed = stream.window(new Duration(2000));
+ JavaTestUtils.attachTestOutputStream(windowed);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testWindowWithSlideDuration() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9),
+ Arrays.asList(10,11,12),
+ Arrays.asList(13,14,15),
+ Arrays.asList(16,17,18));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(1,2,3,4,5,6),
+ Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12),
+ Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
+ Arrays.asList(13,14,15,16,17,18));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000));
+ JavaTestUtils.attachTestOutputStream(windowed);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testTumble() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9),
+ Arrays.asList(10,11,12),
+ Arrays.asList(13,14,15),
+ Arrays.asList(16,17,18));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(1,2,3,4,5,6),
+ Arrays.asList(7,8,9,10,11,12),
+ Arrays.asList(13,14,15,16,17,18));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream windowed = stream.tumble(new Duration(2000));
+ JavaTestUtils.attachTestOutputStream(windowed);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 6, 3);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testFilter() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("giants", "dodgers"),
+ Arrays.asList("yankees", "red socks"));
+
+ List<List<String>> expected = Arrays.asList(
+ Arrays.asList("giants"),
+ Arrays.asList("yankees"));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream filtered = stream.filter(new Function<String, Boolean>() {
+ @Override
+ public Boolean call(String s) throws Exception {
+ return s.contains("a");
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(filtered);
+ List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testGlom() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("giants", "dodgers"),
+ Arrays.asList("yankees", "red socks"));
+
+ List<List<List<String>>> expected = Arrays.asList(
+ Arrays.asList(Arrays.asList("giants", "dodgers")),
+ Arrays.asList(Arrays.asList("yankees", "red socks")));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream glommed = stream.glom();
+ JavaTestUtils.attachTestOutputStream(glommed);
+ List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testMapPartitions() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("giants", "dodgers"),
+ Arrays.asList("yankees", "red socks"));
+
+ List<List<String>> expected = Arrays.asList(
+ Arrays.asList("GIANTSDODGERS"),
+ Arrays.asList("YANKEESRED SOCKS"));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
+ @Override
+ public Iterable<String> call(Iterator<String> in) {
+ String out = "";
+ while (in.hasNext()) {
+ out = out + in.next().toUpperCase();
+ }
+ return Lists.newArrayList(out);
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(mapped);
+ List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ private class IntegerSum extends Function2<Integer, Integer, Integer> {
+ @Override
+ public Integer call(Integer i1, Integer i2) throws Exception {
+ return i1 + i2;
+ }
+ }
+
+ private class IntegerDifference extends Function2<Integer, Integer, Integer> {
+ @Override
+ public Integer call(Integer i1, Integer i2) throws Exception {
+ return i1 - i2;
+ }
+ }
+
+ @Test
+ public void testReduce() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(6),
+ Arrays.asList(15),
+ Arrays.asList(24));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream reduced = stream.reduce(new IntegerSum());
+ JavaTestUtils.attachTestOutputStream(reduced);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testReduceByWindow() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(6),
+ Arrays.asList(21),
+ Arrays.asList(39),
+ Arrays.asList(24));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+ new IntegerDifference(), new Duration(2000), new Duration(1000));
+ JavaTestUtils.attachTestOutputStream(reducedWindowed);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testQueueStream() {
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9));
+
+ JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
+ JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1,2,3));
+ JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4,5,6));
+ JavaRDD<Integer> rdd3 = ssc.sc().parallelize(Arrays.asList(7,8,9));
+
+ LinkedList<JavaRDD<Integer>> rdds = Lists.newLinkedList();
+ rdds.add(rdd1);
+ rdds.add(rdd2);
+ rdds.add(rdd3);
+
+ JavaDStream<Integer> stream = ssc.queueStream(rdds);
+ JavaTestUtils.attachTestOutputStream(stream);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testTransform() {
+ List<List<Integer>> inputData = Arrays.asList(
+ Arrays.asList(1,2,3),
+ Arrays.asList(4,5,6),
+ Arrays.asList(7,8,9));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(3,4,5),
+ Arrays.asList(6,7,8),
+ Arrays.asList(9,10,11));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream transformed = stream.transform(new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+ @Override
+ public JavaRDD<Integer> call(JavaRDD<Integer> in) throws Exception {
+ return in.map(new Function<Integer, Integer>() {
+ @Override
+ public Integer call(Integer i) throws Exception {
+ return i + 2;
+ }
+ });
+ }});
+ JavaTestUtils.attachTestOutputStream(transformed);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testFlatMap() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("go", "giants"),
+ Arrays.asList("boo", "dodgers"),
+ Arrays.asList("athletics"));
+
+ List<List<String>> expected = Arrays.asList(
+ Arrays.asList("g","o","g","i","a","n","t","s"),
+ Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
+ Arrays.asList("a","t","h","l","e","t","i","c","s"));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
+ @Override
+ public Iterable<String> call(String x) {
+ return Lists.newArrayList(x.split("(?!^)"));
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(flatMapped);
+ List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ @Test
+ public void testPairFlatMap() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("giants"),
+ Arrays.asList("dodgers"),
+ Arrays.asList("athletics"));
+
+ List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<Integer, String>(6, "g"),
+ new Tuple2<Integer, String>(6, "i"),
+ new Tuple2<Integer, String>(6, "a"),
+ new Tuple2<Integer, String>(6, "n"),
+ new Tuple2<Integer, String>(6, "t"),
+ new Tuple2<Integer, String>(6, "s")),
+ Arrays.asList(
+ new Tuple2<Integer, String>(7, "d"),
+ new Tuple2<Integer, String>(7, "o"),
+ new Tuple2<Integer, String>(7, "d"),
+ new Tuple2<Integer, String>(7, "g"),
+ new Tuple2<Integer, String>(7, "e"),
+ new Tuple2<Integer, String>(7, "r"),
+ new Tuple2<Integer, String>(7, "s")),
+ Arrays.asList(
+ new Tuple2<Integer, String>(9, "a"),
+ new Tuple2<Integer, String>(9, "t"),
+ new Tuple2<Integer, String>(9, "h"),
+ new Tuple2<Integer, String>(9, "l"),
+ new Tuple2<Integer, String>(9, "e"),
+ new Tuple2<Integer, String>(9, "t"),
+ new Tuple2<Integer, String>(9, "i"),
+ new Tuple2<Integer, String>(9, "c"),
+ new Tuple2<Integer, String>(9, "s")));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
+ @Override
+ public Iterable<Tuple2<Integer, String>> call(String in) throws Exception {
+ List<Tuple2<Integer, String>> out = Lists.newArrayList();
+ for (String letter: in.split("(?!^)")) {
+ out.add(new Tuple2<Integer, String>(in.length(), letter));
+ }
+ return out;
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(flatMapped);
+ List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testUnion() {
+ List<List<Integer>> inputData1 = Arrays.asList(
+ Arrays.asList(1,1),
+ Arrays.asList(2,2),
+ Arrays.asList(3,3));
+
+ List<List<Integer>> inputData2 = Arrays.asList(
+ Arrays.asList(4,4),
+ Arrays.asList(5,5),
+ Arrays.asList(6,6));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(1,1,4,4),
+ Arrays.asList(2,2,5,5),
+ Arrays.asList(3,3,6,6));
+
+ JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
+ JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
+
+ JavaDStream unioned = stream1.union(stream2);
+ JavaTestUtils.attachTestOutputStream(unioned);
+ List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ assertOrderInvariantEquals(expected, result);
+ }
+
+ /*
+ * Performs an order-invariant comparison of lists representing two RDD streams. This allows
+ * us to account for ordering variation within individual RDD's which occurs during windowing.
+ */
+ public static <T extends Comparable> void assertOrderInvariantEquals(
+ List<List<T>> expected, List<List<T>> actual) {
+ for (List<T> list: expected) {
+ Collections.sort(list);
+ }
+ for (List<T> list: actual) {
+ Collections.sort(list);
+ }
+ Assert.assertEquals(expected, actual);
+ }
+
+
+ // PairDStream Functions
+ @Test
+ public void testPairFilter() {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("giants", "dodgers"),
+ Arrays.asList("yankees", "red socks"));
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, Integer>("giants", 6)),
+ Arrays.asList(new Tuple2<String, Integer>("yankees", 7)));
+
+ JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = stream.map(
+ new PairFunction<String, String, Integer>() {
+ @Override
+ public Tuple2 call(String in) throws Exception {
+ return new Tuple2<String, Integer>(in, in.length());
+ }
+ });
+
+ JavaPairDStream<String, Integer> filtered = pairStream.filter(
+ new Function<Tuple2<String, Integer>, Boolean>() {
+ @Override
+ public Boolean call(Tuple2<String, Integer> in) throws Exception {
+ return in._1().contains("a");
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(filtered);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+ new Tuple2<String, String>("california", "giants"),
+ new Tuple2<String, String>("new york", "yankees"),
+ new Tuple2<String, String>("new york", "mets")),
+ Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+ new Tuple2<String, String>("california", "ducks"),
+ new Tuple2<String, String>("new york", "rangers"),
+ new Tuple2<String, String>("new york", "islanders")));
+
+ List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 1),
+ new Tuple2<String, Integer>("california", 3),
+ new Tuple2<String, Integer>("new york", 4),
+ new Tuple2<String, Integer>("new york", 1)),
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 5),
+ new Tuple2<String, Integer>("california", 5),
+ new Tuple2<String, Integer>("new york", 3),
+ new Tuple2<String, Integer>("new york", 1)));
+
+ @Test
+ public void testPairGroupByKey() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, List<String>>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, List<String>>("california", Arrays.asList("dodgers", "giants")),
+ new Tuple2<String, List<String>>("new york", Arrays.asList("yankees", "mets"))),
+ Arrays.asList(
+ new Tuple2<String, List<String>>("california", Arrays.asList("sharks", "ducks")),
+ new Tuple2<String, List<String>>("new york", Arrays.asList("rangers", "islanders"))));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, List<String>> grouped = pairStream.groupByKey();
+ JavaTestUtils.attachTestOutputStream(grouped);
+ List<List<Tuple2<String, List<String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testPairReduceByKey() {
+ List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 4),
+ new Tuple2<String, Integer>("new york", 5)),
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 10),
+ new Tuple2<String, Integer>("new york", 4)));
+
+ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey(new IntegerSum());
+
+ JavaTestUtils.attachTestOutputStream(reduced);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testCombineByKey() {
+ List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 4),
+ new Tuple2<String, Integer>("new york", 5)),
+ Arrays.asList(
+ new Tuple2<String, Integer>("california", 10),
+ new Tuple2<String, Integer>("new york", 4)));
+
+ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Integer> combined = pairStream.<Integer>combineByKey(
+ new Function<Integer, Integer>() {
+ @Override
+ public Integer call(Integer i) throws Exception {
+ return i;
+ }
+ }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2));
+
+ JavaTestUtils.attachTestOutputStream(combined);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testCountByKey() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, Long>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Long>("california", 2L),
+ new Tuple2<String, Long>("new york", 2L)),
+ Arrays.asList(
+ new Tuple2<String, Long>("california", 2L),
+ new Tuple2<String, Long>("new york", 2L)));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Long> counted = pairStream.countByKey();
+ JavaTestUtils.attachTestOutputStream(counted);
+ List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testGroupByKeyAndWindow() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, List<String>>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, List<String>>("california", Arrays.asList("dodgers", "giants")),
+ new Tuple2<String, List<String>>("new york", Arrays.asList("yankees", "mets"))),
+ Arrays.asList(new Tuple2<String, List<String>>("california",
+ Arrays.asList("sharks", "ducks", "dodgers", "giants")),
+ new Tuple2<String, List<String>>("new york", Arrays.asList("rangers", "islanders", "yankees", "mets"))),
+ Arrays.asList(new Tuple2<String, List<String>>("california", Arrays.asList("sharks", "ducks")),
+ new Tuple2<String, List<String>>("new york", Arrays.asList("rangers", "islanders"))));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, List<String>> groupWindowed =
+ pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
+ JavaTestUtils.attachTestOutputStream(groupWindowed);
+ List<List<Tuple2<String, List<String>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testReduceByKeyAndWindow() {
+ List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, Integer>("california", 4),
+ new Tuple2<String, Integer>("new york", 5)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 14),
+ new Tuple2<String, Integer>("new york", 9)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 10),
+ new Tuple2<String, Integer>("new york", 4)));
+
+ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Integer> reduceWindowed =
+ pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000));
+ JavaTestUtils.attachTestOutputStream(reduceWindowed);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testUpdateStateByKey() {
+ List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, Integer>("california", 4),
+ new Tuple2<String, Integer>("new york", 5)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 14),
+ new Tuple2<String, Integer>("new york", 9)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 14),
+ new Tuple2<String, Integer>("new york", 9)));
+
+ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
+ new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>(){
+ @Override
+ public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
+ int out = 0;
+ if (state.isPresent()) {
+ out = out + state.get();
+ }
+ for (Integer v: values) {
+ out = out + v;
+ }
+ return Optional.of(out);
+ }
+ });
+ JavaTestUtils.attachTestOutputStream(updated);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testReduceByKeyAndWindowWithInverse() {
+ List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+ List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, Integer>("california", 4),
+ new Tuple2<String, Integer>("new york", 5)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 14),
+ new Tuple2<String, Integer>("new york", 9)),
+ Arrays.asList(new Tuple2<String, Integer>("california", 10),
+ new Tuple2<String, Integer>("new york", 4)));
+
+ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Integer> reduceWindowed =
+ pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000));
+ JavaTestUtils.attachTestOutputStream(reduceWindowed);
+ List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testCountByKeyAndWindow() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, Long>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Long>("california", 2L),
+ new Tuple2<String, Long>("new york", 2L)),
+ Arrays.asList(
+ new Tuple2<String, Long>("california", 4L),
+ new Tuple2<String, Long>("new york", 4L)),
+ Arrays.asList(
+ new Tuple2<String, Long>("california", 2L),
+ new Tuple2<String, Long>("new york", 2L)));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, Long> counted =
+ pairStream.countByKeyAndWindow(new Duration(2000), new Duration(1000));
+ JavaTestUtils.attachTestOutputStream(counted);
+ List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testMapValues() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, String>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "DODGERS"),
+ new Tuple2<String, String>("california", "GIANTS"),
+ new Tuple2<String, String>("new york", "YANKEES"),
+ new Tuple2<String, String>("new york", "METS")),
+ Arrays.asList(new Tuple2<String, String>("california", "SHARKS"),
+ new Tuple2<String, String>("california", "DUCKS"),
+ new Tuple2<String, String>("new york", "RANGERS"),
+ new Tuple2<String, String>("new york", "ISLANDERS")));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+ JavaPairDStream<String, String> mapped = pairStream.mapValues(new Function<String, String>() {
+ @Override
+ public String call(String s) throws Exception {
+ return s.toUpperCase();
+ }
+ });
+
+ JavaTestUtils.attachTestOutputStream(mapped);
+ List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testFlatMapValues() {
+ List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+ List<List<Tuple2<String, String>>> expected = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "dodgers1"),
+ new Tuple2<String, String>("california", "dodgers2"),
+ new Tuple2<String, String>("california", "giants1"),
+ new Tuple2<String, String>("california", "giants2"),
+ new Tuple2<String, String>("new york", "yankees1"),
+ new Tuple2<String, String>("new york", "yankees2"),
+ new Tuple2<String, String>("new york", "mets1"),
+ new Tuple2<String, String>("new york", "mets2")),
+ Arrays.asList(new Tuple2<String, String>("california", "sharks1"),
+ new Tuple2<String, String>("california", "sharks2"),
+ new Tuple2<String, String>("california", "ducks1"),
+ new Tuple2<String, String>("california", "ducks2"),
+ new Tuple2<String, String>("new york", "rangers1"),
+ new Tuple2<String, String>("new york", "rangers2"),
+ new Tuple2<String, String>("new york", "islanders1"),
+ new Tuple2<String, String>("new york", "islanders2")));
+
+ JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+ ssc, inputData, 1);
+ JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+
+ JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues(
+ new Function<String, Iterable<String>>() {
+ @Override
+ public Iterable<String> call(String in) {
+ List<String> out = new ArrayList<String>();
+ out.add(in + "1");
+ out.add(in + "2");
+ return out;
+ }
+ });
+
+ JavaTestUtils.attachTestOutputStream(flatMapped);
+ List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testCoGroup() {
+ List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+ new Tuple2<String, String>("new york", "yankees")),
+ Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+ new Tuple2<String, String>("new york", "rangers")));
+
+ List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "giants"),
+ new Tuple2<String, String>("new york", "mets")),
+ Arrays.asList(new Tuple2<String, String>("california", "ducks"),
+ new Tuple2<String, String>("new york", "islanders")));
+
+
+ List<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Tuple2<List<String>, List<String>>>("california",
+ new Tuple2<List<String>, List<String>>(Arrays.asList("dodgers"), Arrays.asList("giants"))),
+ new Tuple2<String, Tuple2<List<String>, List<String>>>("new york",
+ new Tuple2<List<String>, List<String>>(Arrays.asList("yankees"), Arrays.asList("mets")))),
+ Arrays.asList(
+ new Tuple2<String, Tuple2<List<String>, List<String>>>("california",
+ new Tuple2<List<String>, List<String>>(Arrays.asList("sharks"), Arrays.asList("ducks"))),
+ new Tuple2<String, Tuple2<List<String>, List<String>>>("new york",
+ new Tuple2<List<String>, List<String>>(Arrays.asList("rangers"), Arrays.asList("islanders")))));
+
+
+ JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+ ssc, stringStringKVStream1, 1);
+ JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+ JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+ ssc, stringStringKVStream2, 1);
+ JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+ JavaPairDStream<String, Tuple2<List<String>, List<String>>> grouped = pairStream1.cogroup(pairStream2);
+ JavaTestUtils.attachTestOutputStream(grouped);
+ List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testJoin() {
+ List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+ new Tuple2<String, String>("new york", "yankees")),
+ Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+ new Tuple2<String, String>("new york", "rangers")));
+
+ List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+ Arrays.asList(new Tuple2<String, String>("california", "giants"),
+ new Tuple2<String, String>("new york", "mets")),
+ Arrays.asList(new Tuple2<String, String>("california", "ducks"),
+ new Tuple2<String, String>("new york", "islanders")));
+
+
+ List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+ Arrays.asList(
+ new Tuple2<String, Tuple2<String, String>>("california",
+ new Tuple2<String, String>("dodgers", "giants")),
+ new Tuple2<String, Tuple2<String, String>>("new york",
+ new Tuple2<String, String>("yankees", "mets"))),
+ Arrays.asList(
+ new Tuple2<String, Tuple2<String, String>>("california",
+ new Tuple2<String, String>("sharks", "ducks")),
+ new Tuple2<String, Tuple2<String, String>>("new york",
+ new Tuple2<String, String>("rangers", "islanders"))));
+
+
+ JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+ ssc, stringStringKVStream1, 1);
+ JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+ JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+ ssc, stringStringKVStream2, 1);
+ JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+ JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.join(pairStream2);
+ JavaTestUtils.attachTestOutputStream(joined);
+ List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+ Assert.assertEquals(expected, result);
+ }
+
+ @Test
+ public void testCheckpointMasterRecovery() throws InterruptedException {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("this", "is"),
+ Arrays.asList("a", "test"),
+ Arrays.asList("counting", "letters"));
+
+ List<List<Integer>> expectedInitial = Arrays.asList(
+ Arrays.asList(4,2));
+ List<List<Integer>> expectedFinal = Arrays.asList(
+ Arrays.asList(1,4),
+ Arrays.asList(8,7));
+
+
+ File tempDir = Files.createTempDir();
+ ssc.checkpoint(tempDir.getAbsolutePath(), new Duration(1000));
+
+ JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ @Override
+ public Integer call(String s) throws Exception {
+ return s.length();
+ }
+ });
+ JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+ List<List<Integer>> initialResult = JavaTestUtils.runStreams(ssc, 1, 1);
+
+ assertOrderInvariantEquals(expectedInitial, initialResult);
+ Thread.sleep(1000);
+
+ ssc.stop();
+ ssc = new JavaStreamingContext(tempDir.getAbsolutePath());
+ ssc.start();
+ List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 2);
+ assertOrderInvariantEquals(expectedFinal, finalResult);
+ }
+
+ /** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD
+ @Test
+ public void testCheckpointofIndividualStream() throws InterruptedException {
+ List<List<String>> inputData = Arrays.asList(
+ Arrays.asList("this", "is"),
+ Arrays.asList("a", "test"),
+ Arrays.asList("counting", "letters"));
+
+ List<List<Integer>> expected = Arrays.asList(
+ Arrays.asList(4,2),
+ Arrays.asList(1,4),
+ Arrays.asList(8,7));
+
+ JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ @Override
+ public Integer call(String s) throws Exception {
+ return s.length();
+ }
+ });
+ JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+
+ letterCount.checkpoint(new Duration(1000));
+
+ List<List<Integer>> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3);
+ assertOrderInvariantEquals(expected, result1);
+ }
+ */
+
+ // Input stream tests. These mostly just test that we can instantiate a given InputStream with
+ // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
+ // InputStream functionality is deferred to the existing Scala tests.
+ @Test
+ public void testKafkaStream() {
+ HashMap<String, Integer> topics = Maps.newHashMap();
+ HashMap<KafkaPartitionKey, Long> offsets = Maps.newHashMap();
+ JavaDStream test1 = ssc.kafkaStream("localhost", 12345, "group", topics);
+ JavaDStream test2 = ssc.kafkaStream("localhost", 12345, "group", topics, offsets);
+ JavaDStream test3 = ssc.kafkaStream("localhost", 12345, "group", topics, offsets,
+ StorageLevel.MEMORY_AND_DISK());
+ }
+
+ @Test
+ public void testNetworkTextStream() {
+ JavaDStream test = ssc.networkTextStream("localhost", 12345);
+ }
+
+ @Test
+ public void testNetworkString() {
+ class Converter extends Function<InputStream, Iterable<String>> {
+ public Iterable<String> call(InputStream in) {
+ BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+ List<String> out = new ArrayList<String>();
+ try {
+ while (true) {
+ String line = reader.readLine();
+ if (line == null) { break; }
+ out.add(line);
+ }
+ } catch (IOException e) { }
+ return out;
+ }
+ }
+
+ JavaDStream test = ssc.networkStream(
+ "localhost",
+ 12345,
+ new Converter(),
+ StorageLevel.MEMORY_ONLY());
+ }
+
+ @Test
+ public void testTextFileStream() {
+ JavaDStream test = ssc.textFileStream("/tmp/foo");
+ }
+
+ @Test
+ public void testRawNetworkStream() {
+ JavaDStream test = ssc.rawNetworkStream("localhost", 12345);
+ }
+
+ @Test
+ public void testFlumeStream() {
+ JavaDStream test = ssc.flumeStream("localhost", 12345);
+ }
+
+ @Test
+ public void testFileStream() {
+ JavaPairDStream<String, String> foo =
+ ssc.<String, String, SequenceFileInputFormat>fileStream("/tmp/foo");
+ }
+}
diff --git a/streaming/src/test/java/JavaTestUtils.scala b/streaming/src/test/java/JavaTestUtils.scala
new file mode 100644
index 0000000000..56349837e5
--- /dev/null
+++ b/streaming/src/test/java/JavaTestUtils.scala
@@ -0,0 +1,65 @@
+package spark.streaming
+
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import java.util.{List => JList}
+import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
+import spark.streaming._
+import java.util.ArrayList
+import collection.JavaConversions._
+
+/** Exposes streaming test functionality in a Java-friendly way. */
+trait JavaTestBase extends TestSuiteBase {
+
+ /**
+ * Create a [[spark.streaming.TestInputStream]] and attach it to the supplied context.
+ * The stream will be derived from the supplied lists of Java objects.
+ **/
+ def attachTestInputStream[T](
+ ssc: JavaStreamingContext,
+ data: JList[JList[T]],
+ numPartitions: Int) = {
+ val seqData = data.map(Seq(_:_*))
+
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions)
+ ssc.ssc.registerInputStream(dstream)
+ new JavaDStream[T](dstream)
+ }
+
+ /**
+ * Attach a provided stream to it's associated StreamingContext as a
+ * [[spark.streaming.TestOutputStream]].
+ **/
+ def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T,This]](
+ dstream: JavaDStreamLike[T, This]) = {
+ implicit val cm: ClassManifest[T] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ val ostream = new TestOutputStream(dstream.dstream,
+ new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]])
+ dstream.dstream.ssc.registerOutputStream(ostream)
+ }
+
+ /**
+ * Process all registered streams for a numBatches batches, failing if
+ * numExpectedOutput RDD's are not generated. Generated RDD's are collected
+ * and returned, represented as a list for each batch interval.
+ */
+ def runStreams[V](
+ ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
+ implicit val cm: ClassManifest[V] =
+ implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
+ val out = new ArrayList[JList[V]]()
+ res.map(entry => out.append(new ArrayList[V](entry)))
+ out
+ }
+}
+
+object JavaTestUtils extends JavaTestBase {
+
+}
+
+object JavaCheckpointTestUtils extends JavaTestBase {
+ override def actuallyWait = true
+} \ No newline at end of file
diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..edfa1243fa
--- /dev/null
+++ b/streaming/src/test/resources/log4j.properties
@@ -0,0 +1,11 @@
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
new file mode 100644
index 0000000000..bfdf32c73e
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -0,0 +1,218 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import scala.runtime.RichInt
+import util.ManualClock
+
+class BasicOperationsSuite extends TestSuiteBase {
+
+ override def framework() = "BasicOperationsSuite"
+
+ after {
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ test("map") {
+ val input = Seq(1 to 4, 5 to 8, 9 to 12)
+ testOperation(
+ input,
+ (r: DStream[Int]) => r.map(_.toString),
+ input.map(_.map(_.toString))
+ )
+ }
+
+ test("flatmap") {
+ val input = Seq(1 to 4, 5 to 8, 9 to 12)
+ testOperation(
+ input,
+ (r: DStream[Int]) => r.flatMap(x => Seq(x, x * 2)),
+ input.map(_.flatMap(x => Array(x, x * 2)))
+ )
+ }
+
+ test("filter") {
+ val input = Seq(1 to 4, 5 to 8, 9 to 12)
+ testOperation(
+ input,
+ (r: DStream[Int]) => r.filter(x => (x % 2 == 0)),
+ input.map(_.filter(x => (x % 2 == 0)))
+ )
+ }
+
+ test("glom") {
+ assert(numInputPartitions === 2, "Number of input partitions has been changed from 2")
+ val input = Seq(1 to 4, 5 to 8, 9 to 12)
+ val output = Seq(
+ Seq( Seq(1, 2), Seq(3, 4) ),
+ Seq( Seq(5, 6), Seq(7, 8) ),
+ Seq( Seq(9, 10), Seq(11, 12) )
+ )
+ val operation = (r: DStream[Int]) => r.glom().map(_.toSeq)
+ testOperation(input, operation, output)
+ }
+
+ test("mapPartitions") {
+ assert(numInputPartitions === 2, "Number of input partitions has been changed from 2")
+ val input = Seq(1 to 4, 5 to 8, 9 to 12)
+ val output = Seq(Seq(3, 7), Seq(11, 15), Seq(19, 23))
+ val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.reduce(_ + _)))
+ testOperation(input, operation, output, true)
+ }
+
+ test("groupByKey") {
+ testOperation(
+ Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+ (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(),
+ Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ),
+ true
+ )
+ }
+
+ test("reduceByKey") {
+ testOperation(
+ Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+ (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _),
+ Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ),
+ true
+ )
+ }
+
+ test("reduce") {
+ testOperation(
+ Seq(1 to 4, 5 to 8, 9 to 12),
+ (s: DStream[Int]) => s.reduce(_ + _),
+ Seq(Seq(10), Seq(26), Seq(42))
+ )
+ }
+
+ test("mapValues") {
+ testOperation(
+ Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+ (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).mapValues(_ + 10),
+ Seq( Seq(("a", 12), ("b", 11)), Seq(("", 12)), Seq() ),
+ true
+ )
+ }
+
+ test("flatMapValues") {
+ testOperation(
+ Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+ (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)),
+ Seq( Seq(("a", 2), ("a", 12), ("b", 1), ("b", 11)), Seq(("", 2), ("", 12)), Seq() ),
+ true
+ )
+ }
+
+ test("cogroup") {
+ val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() )
+ val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq() )
+ val outputData = Seq(
+ Seq( ("a", (Seq(1, 1), Seq("x", "x"))), ("b", (Seq(1), Seq("x"))) ),
+ Seq( ("a", (Seq(1), Seq())), ("b", (Seq(), Seq("x"))), ("", (Seq(1), Seq("x"))) ),
+ Seq( ("", (Seq(1), Seq())) ),
+ Seq( )
+ )
+ val operation = (s1: DStream[String], s2: DStream[String]) => {
+ s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x")))
+ }
+ testOperation(inputData1, inputData2, operation, outputData, true)
+ }
+
+ test("join") {
+ val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() )
+ val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("") )
+ val outputData = Seq(
+ Seq( ("a", (1, "x")), ("b", (1, "x")) ),
+ Seq( ("", (1, "x")) ),
+ Seq( ),
+ Seq( )
+ )
+ val operation = (s1: DStream[String], s2: DStream[String]) => {
+ s1.map(x => (x,1)).join(s2.map(x => (x,"x")))
+ }
+ testOperation(inputData1, inputData2, operation, outputData, true)
+ }
+
+ test("updateStateByKey") {
+ val inputData =
+ Seq(
+ Seq("a"),
+ Seq("a", "b"),
+ Seq("a", "b", "c"),
+ Seq("a", "b"),
+ Seq("a"),
+ Seq()
+ )
+
+ val outputData =
+ Seq(
+ Seq(("a", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 3), ("b", 2), ("c", 1)),
+ Seq(("a", 4), ("b", 3), ("c", 1)),
+ Seq(("a", 5), ("b", 3), ("c", 1)),
+ Seq(("a", 5), ("b", 3), ("c", 1))
+ )
+
+ val updateStateOperation = (s: DStream[String]) => {
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some(values.foldLeft(0)(_ + _) + state.getOrElse(0))
+ }
+ s.map(x => (x, 1)).updateStateByKey[Int](updateFunc)
+ }
+
+ testOperation(inputData, updateStateOperation, outputData, true)
+ }
+
+ test("forgetting of RDDs - map and window operations") {
+ assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second")
+
+ val input = (0 until 10).map(x => Seq(x, x + 1)).toSeq
+ val rememberDuration = Seconds(3)
+
+ assert(input.size === 10, "Number of inputs have changed")
+
+ def operation(s: DStream[Int]): DStream[(Int, Int)] = {
+ s.map(x => (x % 10, 1))
+ .window(Seconds(2), Seconds(1))
+ .window(Seconds(4), Seconds(2))
+ }
+
+ val ssc = setupStreams(input, operation _)
+ ssc.remember(rememberDuration)
+ runStreams[(Int, Int)](ssc, input.size, input.size / 2)
+
+ val windowedStream2 = ssc.graph.getOutputStreams().head.dependencies.head
+ val windowedStream1 = windowedStream2.dependencies.head
+ val mappedStream = windowedStream1.dependencies.head
+
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ assert(clock.time === Seconds(10).milliseconds)
+
+ // IDEALLY
+ // WindowedStream2 should remember till 7 seconds: 10, 8,
+ // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5
+ // MappedStream should remember till 7 seconds: 10, 9, 8, 7, 6, 5, 4, 3,
+
+ // IN THIS TEST
+ // WindowedStream2 should remember till 7 seconds: 10, 8,
+ // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5, 4
+ // MappedStream should remember till 7 seconds: 10, 9, 8, 7, 6, 5, 4, 3, 2
+
+ // WindowedStream2
+ assert(windowedStream2.generatedRDDs.contains(Time(10000)))
+ assert(windowedStream2.generatedRDDs.contains(Time(8000)))
+ assert(!windowedStream2.generatedRDDs.contains(Time(6000)))
+
+ // WindowedStream1
+ assert(windowedStream1.generatedRDDs.contains(Time(10000)))
+ assert(windowedStream1.generatedRDDs.contains(Time(4000)))
+ assert(!windowedStream1.generatedRDDs.contains(Time(3000)))
+
+ // MappedStream
+ assert(mappedStream.generatedRDDs.contains(Time(10000)))
+ assert(mappedStream.generatedRDDs.contains(Time(2000)))
+ assert(!mappedStream.generatedRDDs.contains(Time(1000)))
+ }
+}
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
new file mode 100644
index 0000000000..d2f32c189b
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -0,0 +1,210 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import java.io.File
+import runtime.RichInt
+import org.scalatest.BeforeAndAfter
+import org.apache.commons.io.FileUtils
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import util.{Clock, ManualClock}
+
+class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
+
+ before {
+ FileUtils.deleteDirectory(new File(checkpointDir))
+ }
+
+ after {
+ if (ssc != null) ssc.stop()
+ FileUtils.deleteDirectory(new File(checkpointDir))
+
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ var ssc: StreamingContext = null
+
+ override def framework = "CheckpointSuite"
+
+ override def batchDuration = Milliseconds(500)
+
+ override def checkpointInterval = batchDuration
+
+ override def actuallyWait = true
+
+ test("basic stream+rdd recovery") {
+
+ assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
+ assert(checkpointInterval === batchDuration, "checkpointInterval for this test much be same as batchDuration")
+
+ System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+ val stateStreamCheckpointInterval = Seconds(1)
+
+ // this ensure checkpointing occurs at least once
+ val firstNumBatches = (stateStreamCheckpointInterval / batchDuration) * 2
+ val secondNumBatches = firstNumBatches
+
+ // Setup the streams
+ val input = (1 to 10).map(_ => Seq("a")).toSeq
+ val operation = (st: DStream[String]) => {
+ val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
+ Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ }
+ st.map(x => (x, 1))
+ .updateStateByKey[RichInt](updateFunc)
+ .checkpoint(stateStreamCheckpointInterval)
+ .map(t => (t._1, t._2.self))
+ }
+ var ssc = setupStreams(input, operation)
+ var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+
+ // Run till a time such that at least one RDD in the stream should have been checkpointed,
+ // then check whether some RDD has been checkpointed or not
+ ssc.start()
+ runStreamsWithRealDelay(ssc, firstNumBatches)
+ logInfo("Checkpoint data of state stream = \n[" + stateStream.checkpointData.rdds.mkString(",\n") + "]")
+ assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before first failure")
+ stateStream.checkpointData.rdds.foreach {
+ case (time, data) => {
+ val file = new File(data.toString)
+ assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist")
+ }
+ }
+
+ // Run till a further time such that previous checkpoint files in the stream would be deleted
+ // and check whether the earlier checkpoint files are deleted
+ val checkpointFiles = stateStream.checkpointData.rdds.map(x => new File(x._2.toString))
+ runStreamsWithRealDelay(ssc, secondNumBatches)
+ checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
+ ssc.stop()
+
+ // Restart stream computation using the checkpoint file and check whether
+ // checkpointed RDDs have been restored or not
+ ssc = new StreamingContext(checkpointDir)
+ stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+ logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
+ assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure")
+
+
+ // Run one batch to generate a new checkpoint file and check whether some RDD
+ // is present in the checkpoint data or not
+ ssc.start()
+ runStreamsWithRealDelay(ssc, 1)
+ assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before second failure")
+ stateStream.checkpointData.rdds.foreach {
+ case (time, data) => {
+ val file = new File(data.toString)
+ assert(file.exists(),
+ "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist")
+ }
+ }
+ ssc.stop()
+
+ // Restart stream computation from the new checkpoint file to see whether that file has
+ // correct checkpoint data
+ ssc = new StreamingContext(checkpointDir)
+ stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+ logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
+ assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure")
+
+ // Adjust manual clock time as if it is being restarted after a delay
+ System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
+ ssc.start()
+ runStreamsWithRealDelay(ssc, 4)
+ ssc.stop()
+ System.clearProperty("spark.streaming.manualClock.jump")
+ ssc = null
+ }
+
+ test("map and reduceByKey") {
+ testCheckpointedOperation(
+ Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ),
+ (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _),
+ Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ),
+ 3
+ )
+ }
+
+ test("reduceByKeyAndWindowInv") {
+ val n = 10
+ val w = 4
+ val input = (1 to n).map(_ => Seq("a")).toSeq
+ val output = Seq(Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4)))
+ val operation = (st: DStream[String]) => {
+ st.map(x => (x, 1))
+ .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration)
+ .checkpoint(batchDuration * 2)
+ }
+ testCheckpointedOperation(input, operation, output, 7)
+ }
+
+ test("updateStateByKey") {
+ val input = (1 to 10).map(_ => Seq("a")).toSeq
+ val output = (1 to 10).map(x => Seq(("a", x))).toSeq
+ val operation = (st: DStream[String]) => {
+ val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
+ Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ }
+ st.map(x => (x, 1))
+ .updateStateByKey[RichInt](updateFunc)
+ .checkpoint(batchDuration * 2)
+ .map(t => (t._1, t._2.self))
+ }
+ testCheckpointedOperation(input, operation, output, 7)
+ }
+
+ /**
+ * Tests a streaming operation under checkpointing, by restart the operation
+ * from checkpoint file and verifying whether the final output is correct.
+ * The output is assumed to have come from a reliable queue which an replay
+ * data as required.
+ */
+ def testCheckpointedOperation[U: ClassManifest, V: ClassManifest](
+ input: Seq[Seq[U]],
+ operation: DStream[U] => DStream[V],
+ expectedOutput: Seq[Seq[V]],
+ initialNumBatches: Int
+ ) {
+
+ // Current code assumes that:
+ // number of inputs = number of outputs = number of batches to be run
+ val totalNumBatches = input.size
+ val nextNumBatches = totalNumBatches - initialNumBatches
+ val initialNumExpectedOutputs = initialNumBatches
+ val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs
+
+ // Do the computation for initial number of batches, create checkpoint file and quit
+ ssc = setupStreams[U, V](input, operation)
+ val output = runStreams[V](ssc, initialNumBatches, initialNumExpectedOutputs)
+ verifyOutput[V](output, expectedOutput.take(initialNumBatches), true)
+ Thread.sleep(1000)
+
+ // Restart and complete the computation from checkpoint file
+ logInfo(
+ "\n-------------------------------------------\n" +
+ " Restarting stream computation " +
+ "\n-------------------------------------------\n"
+ )
+ ssc = new StreamingContext(checkpointDir)
+ val outputNew = runStreams[V](ssc, nextNumBatches, nextNumExpectedOutputs)
+ verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true)
+ ssc = null
+ }
+
+ /**
+ * Advances the manual clock on the streaming scheduler by given number of batches.
+ * It also wait for the expected amount of time for each batch.
+ */
+ def runStreamsWithRealDelay(ssc: StreamingContext, numBatches: Long) {
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ logInfo("Manual clock before advancing = " + clock.time)
+ for (i <- 1 to numBatches.toInt) {
+ clock.addToTime(batchDuration.milliseconds)
+ Thread.sleep(batchDuration.milliseconds)
+ }
+ logInfo("Manual clock after advancing = " + clock.time)
+ Thread.sleep(batchDuration.milliseconds)
+ }
+
+} \ No newline at end of file
diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
new file mode 100644
index 0000000000..7493ac1207
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
@@ -0,0 +1,191 @@
+package spark.streaming
+
+import org.scalatest.BeforeAndAfter
+import org.apache.commons.io.FileUtils
+import java.io.File
+import scala.runtime.RichInt
+import scala.util.Random
+import spark.streaming.StreamingContext._
+import collection.mutable.ArrayBuffer
+import spark.Logging
+
+/**
+ * This testsuite tests master failures at random times while the stream is running using
+ * the real clock.
+ */
+class FailureSuite extends TestSuiteBase with BeforeAndAfter {
+
+ before {
+ FileUtils.deleteDirectory(new File(checkpointDir))
+ }
+
+ after {
+ FailureSuite.reset()
+ FileUtils.deleteDirectory(new File(checkpointDir))
+
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ override def framework = "CheckpointSuite"
+
+ override def batchDuration = Milliseconds(500)
+
+ override def checkpointDir = "checkpoint"
+
+ override def checkpointInterval = batchDuration
+
+ test("multiple failures with updateStateByKey") {
+ val n = 30
+ // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
+ val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq
+ // Last output: [ (a, 465) ] for n=30
+ val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) )
+
+ val operation = (st: DStream[String]) => {
+ val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
+ Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ }
+ st.map(x => (x, 1))
+ .updateStateByKey[RichInt](updateFunc)
+ .checkpoint(Seconds(2))
+ .map(t => (t._1, t._2.self))
+ }
+
+ testOperationWithMultipleFailures(input, operation, lastOutput, n, n)
+ }
+
+ test("multiple failures with reduceByKeyAndWindow") {
+ val n = 30
+ val w = 100
+ assert(w > n, "Window should be much larger than the number of input sets in this test")
+ // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
+ val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq
+ // Last output: [ (a, 465) ]
+ val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) )
+
+ val operation = (st: DStream[String]) => {
+ st.map(x => (x, 1))
+ .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration)
+ .checkpoint(Seconds(2))
+ }
+
+ testOperationWithMultipleFailures(input, operation, lastOutput, n, n)
+ }
+
+
+ /**
+ * Tests stream operation with multiple master failures, and verifies whether the
+ * final set of output values is as expected or not. Checking the final value is
+ * proof that no intermediate data was lost due to master failures.
+ */
+ def testOperationWithMultipleFailures[U: ClassManifest, V: ClassManifest](
+ input: Seq[Seq[U]],
+ operation: DStream[U] => DStream[V],
+ lastExpectedOutput: Seq[V],
+ numBatches: Int,
+ numExpectedOutput: Int
+ ) {
+ var ssc = setupStreams[U, V](input, operation)
+ val mergedOutput = new ArrayBuffer[Seq[V]]()
+
+ var totalTimeRan = 0L
+ while(totalTimeRan <= numBatches * batchDuration.milliseconds * 2) {
+ new KillingThread(ssc, numBatches * batchDuration.milliseconds.toInt / 4).start()
+ val (output, timeRan) = runStreamsWithRealClock[V](ssc, numBatches, numExpectedOutput)
+
+ mergedOutput ++= output
+ totalTimeRan += timeRan
+ logInfo("New output = " + output)
+ logInfo("Merged output = " + mergedOutput)
+ logInfo("Total time spent = " + totalTimeRan)
+ val sleepTime = Random.nextInt(numBatches * batchDuration.milliseconds.toInt / 8)
+ logInfo(
+ "\n-------------------------------------------\n" +
+ " Restarting stream computation in " + sleepTime + " ms " +
+ "\n-------------------------------------------\n"
+ )
+ Thread.sleep(sleepTime)
+ FailureSuite.failed = false
+ ssc = new StreamingContext(checkpointDir)
+ }
+ ssc.stop()
+ ssc = null
+
+ // Verify whether the last output is the expected one
+ val lastOutput = mergedOutput(mergedOutput.lastIndexWhere(!_.isEmpty))
+ assert(lastOutput.toSet === lastExpectedOutput.toSet)
+ logInfo("Finished computation after " + FailureSuite.failureCount + " failures")
+ }
+
+ /**
+ * Runs the streams set up in `ssc` on real clock until the expected max number of
+ */
+ def runStreamsWithRealClock[V: ClassManifest](
+ ssc: StreamingContext,
+ numBatches: Int,
+ maxExpectedOutput: Int
+ ): (Seq[Seq[V]], Long) = {
+
+ System.clearProperty("spark.streaming.clock")
+
+ assert(numBatches > 0, "Number of batches to run stream computation is zero")
+ assert(maxExpectedOutput > 0, "Max expected outputs after " + numBatches + " is zero")
+ logInfo("numBatches = " + numBatches + ", maxExpectedOutput = " + maxExpectedOutput)
+
+ // Get the output buffer
+ val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
+ val output = outputStream.output
+ val waitTime = (batchDuration.milliseconds * (numBatches.toDouble + 0.5)).toLong
+ val startTime = System.currentTimeMillis()
+
+ try {
+ // Start computation
+ ssc.start()
+
+ // Wait until expected number of output items have been generated
+ while (output.size < maxExpectedOutput && System.currentTimeMillis() - startTime < waitTime && !FailureSuite.failed) {
+ logInfo("output.size = " + output.size + ", maxExpectedOutput = " + maxExpectedOutput)
+ Thread.sleep(100)
+ }
+ } catch {
+ case e: Exception => logInfo("Exception while running streams: " + e)
+ } finally {
+ ssc.stop()
+ }
+ val timeTaken = System.currentTimeMillis() - startTime
+ logInfo("" + output.size + " sets of output generated in " + timeTaken + " ms")
+ (output, timeTaken)
+ }
+
+
+}
+
+object FailureSuite {
+ var failed = false
+ var failureCount = 0
+
+ def reset() {
+ failed = false
+ failureCount = 0
+ }
+}
+
+class KillingThread(ssc: StreamingContext, maxKillWaitTime: Int) extends Thread with Logging {
+ initLogging()
+
+ override def run() {
+ var minKillWaitTime = if (FailureSuite.failureCount == 0) 3000 else 1000 // to allow the first checkpoint
+ val killWaitTime = minKillWaitTime + Random.nextInt(maxKillWaitTime)
+ logInfo("Kill wait time = " + killWaitTime)
+ Thread.sleep(killWaitTime.toLong)
+ logInfo(
+ "\n---------------------------------------\n" +
+ "Killing streaming context after " + killWaitTime + " ms" +
+ "\n---------------------------------------\n"
+ )
+ if (ssc != null) ssc.stop()
+ FailureSuite.failed = true
+ FailureSuite.failureCount += 1
+ }
+}
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
new file mode 100644
index 0000000000..d7ba7a5d17
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -0,0 +1,355 @@
+package spark.streaming
+
+import dstream.SparkFlumeEvent
+import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
+import java.io.{File, BufferedWriter, OutputStreamWriter}
+import java.util.concurrent.{TimeUnit, ArrayBlockingQueue}
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import util.ManualClock
+import spark.storage.StorageLevel
+import spark.Logging
+import scala.util.Random
+import org.apache.commons.io.FileUtils
+import org.scalatest.BeforeAndAfter
+import org.apache.flume.source.avro.AvroSourceProtocol
+import org.apache.flume.source.avro.AvroFlumeEvent
+import org.apache.flume.source.avro.Status
+import org.apache.avro.ipc.{specific, NettyTransceiver}
+import org.apache.avro.ipc.specific.SpecificRequestor
+import java.nio.ByteBuffer
+import collection.JavaConversions._
+import java.nio.charset.Charset
+
+class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
+
+ System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+ val testPort = 9999
+ var testServer: TestServer = null
+ var testDir: File = null
+
+ override def checkpointDir = "checkpoint"
+
+ after {
+ FileUtils.deleteDirectory(new File(checkpointDir))
+ if (testServer != null) {
+ testServer.stop()
+ testServer = null
+ }
+ if (testDir != null && testDir.exists()) {
+ FileUtils.deleteDirectory(testDir)
+ testDir = null
+ }
+
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ test("network input stream") {
+ // Start the server
+ testServer = new TestServer(testPort)
+ testServer.start()
+
+ // Set up the streaming context and input streams
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]]
+ val outputStream = new TestOutputStream(networkStream, outputBuffer)
+ def output = outputBuffer.flatMap(x => x)
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ // Feed data to the server to send to the network receiver
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5)
+ val expectedOutput = input.map(_.toString)
+ Thread.sleep(1000)
+ for (i <- 0 until input.size) {
+ testServer.send(input(i).toString + "\n")
+ Thread.sleep(500)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+ Thread.sleep(1000)
+ logInfo("Stopping server")
+ testServer.stop()
+ logInfo("Stopping context")
+ ssc.stop()
+
+ // Verify whether data received was as expected
+ logInfo("--------------------------------")
+ logInfo("output.size = " + outputBuffer.size)
+ logInfo("output")
+ outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("expected output.size = " + expectedOutput.size)
+ logInfo("expected output")
+ expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("--------------------------------")
+
+ // Verify whether all the elements received are as expected
+ // (whether the elements were received one in each interval is not verified)
+ assert(output.size === expectedOutput.size)
+ for (i <- 0 until output.size) {
+ assert(output(i) === expectedOutput(i))
+ }
+ }
+
+ test("network input stream with checkpoint") {
+ // Start the server
+ testServer = new TestServer(testPort)
+ testServer.start()
+
+ // Set up the streaming context and input streams
+ var ssc = new StreamingContext(master, framework, batchDuration)
+ ssc.checkpoint(checkpointDir, checkpointInterval)
+ val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+ var outputStream = new TestOutputStream(networkStream, new ArrayBuffer[Seq[String]])
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ // Feed data to the server to send to the network receiver
+ var clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ for (i <- Seq(1, 2, 3)) {
+ testServer.send(i.toString + "\n")
+ Thread.sleep(100)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+ Thread.sleep(500)
+ assert(outputStream.output.size > 0)
+ ssc.stop()
+
+ // Restart stream computation from checkpoint and feed more data to see whether
+ // they are being received and processed
+ logInfo("*********** RESTARTING ************")
+ ssc = new StreamingContext(checkpointDir)
+ ssc.start()
+ clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ for (i <- Seq(4, 5, 6)) {
+ testServer.send(i.toString + "\n")
+ Thread.sleep(100)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+ Thread.sleep(500)
+ outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]]
+ assert(outputStream.output.size > 0)
+ ssc.stop()
+ }
+
+ test("flume input stream") {
+ // Set up the streaming context and input streams
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val flumeStream = ssc.flumeStream("localhost", 33333, StorageLevel.MEMORY_AND_DISK)
+ val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
+ with SynchronizedBuffer[Seq[SparkFlumeEvent]]
+ val outputStream = new TestOutputStream(flumeStream, outputBuffer)
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5)
+
+ val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", 33333));
+ val client = SpecificRequestor.getClient(
+ classOf[AvroSourceProtocol], transceiver);
+
+ for (i <- 0 until input.size) {
+ val event = new AvroFlumeEvent
+ event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
+ event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
+ client.append(event)
+ Thread.sleep(500)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+
+ val startTime = System.currentTimeMillis()
+ while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+ logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
+ Thread.sleep(100)
+ }
+ Thread.sleep(1000)
+ val timeTaken = System.currentTimeMillis() - startTime
+ assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+ logInfo("Stopping context")
+ ssc.stop()
+
+ val decoder = Charset.forName("UTF-8").newDecoder()
+
+ assert(outputBuffer.size === input.length)
+ for (i <- 0 until outputBuffer.size) {
+ assert(outputBuffer(i).size === 1)
+ val str = decoder.decode(outputBuffer(i).head.event.getBody)
+ assert(str.toString === input(i).toString)
+ assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
+ }
+ }
+
+ test("file input stream") {
+
+ // Create a temporary directory
+ testDir = {
+ var temp = File.createTempFile(".temp.", Random.nextInt().toString)
+ temp.delete()
+ temp.mkdirs()
+ logInfo("Created temp dir " + temp)
+ temp
+ }
+
+ // Set up the streaming context and input streams
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val filestream = ssc.textFileStream(testDir.toString)
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
+ def output = outputBuffer.flatMap(x => x)
+ val outputStream = new TestOutputStream(filestream, outputBuffer)
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ // Create files in the temporary directory so that Spark Streaming can read data from it
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5)
+ val expectedOutput = input.map(_.toString)
+ Thread.sleep(1000)
+ for (i <- 0 until input.size) {
+ FileUtils.writeStringToFile(new File(testDir, i.toString), input(i).toString + "\n")
+ Thread.sleep(500)
+ clock.addToTime(batchDuration.milliseconds)
+ //Thread.sleep(100)
+ }
+ val startTime = System.currentTimeMillis()
+ /*while (output.size < expectedOutput.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+ logInfo("output.size = " + output.size + ", expectedOutput.size = " + expectedOutput.size)
+ Thread.sleep(100)
+ }*/
+ Thread.sleep(1000)
+ val timeTaken = System.currentTimeMillis() - startTime
+ assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+ logInfo("Stopping context")
+ ssc.stop()
+
+ // Verify whether data received by Spark Streaming was as expected
+ logInfo("--------------------------------")
+ logInfo("output.size = " + outputBuffer.size)
+ logInfo("output")
+ outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("expected output.size = " + expectedOutput.size)
+ logInfo("expected output")
+ expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("--------------------------------")
+
+ // Verify whether all the elements received are as expected
+ // (whether the elements were received one in each interval is not verified)
+ assert(output.size === expectedOutput.size)
+ for (i <- 0 until output.size) {
+ assert(output(i).size === 1)
+ assert(output(i).head.toString === expectedOutput(i))
+ }
+ }
+
+ test("file input stream with checkpoint") {
+ // Create a temporary directory
+ testDir = {
+ var temp = File.createTempFile(".temp.", Random.nextInt().toString)
+ temp.delete()
+ temp.mkdirs()
+ logInfo("Created temp dir " + temp)
+ temp
+ }
+
+ // Set up the streaming context and input streams
+ var ssc = new StreamingContext(master, framework, batchDuration)
+ ssc.checkpoint(checkpointDir, checkpointInterval)
+ val filestream = ssc.textFileStream(testDir.toString)
+ var outputStream = new TestOutputStream(filestream, new ArrayBuffer[Seq[String]])
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ // Create files and advance manual clock to process them
+ var clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ Thread.sleep(1000)
+ for (i <- Seq(1, 2, 3)) {
+ FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
+ Thread.sleep(100)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+ Thread.sleep(500)
+ logInfo("Output = " + outputStream.output.mkString(","))
+ assert(outputStream.output.size > 0)
+ ssc.stop()
+
+ // Restart stream computation from checkpoint and create more files to see whether
+ // they are being processed
+ logInfo("*********** RESTARTING ************")
+ ssc = new StreamingContext(checkpointDir)
+ ssc.start()
+ clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ Thread.sleep(500)
+ for (i <- Seq(4, 5, 6)) {
+ FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
+ Thread.sleep(100)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+ Thread.sleep(500)
+ outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]]
+ logInfo("Output = " + outputStream.output.mkString(","))
+ assert(outputStream.output.size > 0)
+ ssc.stop()
+ }
+}
+
+
+class TestServer(port: Int) extends Logging {
+
+ val queue = new ArrayBlockingQueue[String](100)
+
+ val serverSocket = new ServerSocket(port)
+
+ val servingThread = new Thread() {
+ override def run() {
+ try {
+ while(true) {
+ logInfo("Accepting connections on port " + port)
+ val clientSocket = serverSocket.accept()
+ logInfo("New connection")
+ try {
+ clientSocket.setTcpNoDelay(true)
+ val outputStream = new BufferedWriter(new OutputStreamWriter(clientSocket.getOutputStream))
+
+ while(clientSocket.isConnected) {
+ val msg = queue.poll(100, TimeUnit.MILLISECONDS)
+ if (msg != null) {
+ outputStream.write(msg)
+ outputStream.flush()
+ logInfo("Message '" + msg + "' sent")
+ }
+ }
+ } catch {
+ case e: SocketException => logError("TestServer error", e)
+ } finally {
+ logInfo("Connection closed")
+ if (!clientSocket.isClosed) clientSocket.close()
+ }
+ }
+ } catch {
+ case ie: InterruptedException =>
+
+ } finally {
+ serverSocket.close()
+ }
+ }
+ }
+
+ def start() { servingThread.start() }
+
+ def send(msg: String) { queue.add(msg) }
+
+ def stop() { servingThread.interrupt() }
+}
+
+object TestServer {
+ def main(args: Array[String]) {
+ val s = new TestServer(9999)
+ s.start()
+ while(true) {
+ Thread.sleep(1000)
+ s.send("hello")
+ }
+ }
+}
diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
new file mode 100644
index 0000000000..49129f3964
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
@@ -0,0 +1,291 @@
+package spark.streaming
+
+import spark.streaming.dstream.{InputDStream, ForEachDStream}
+import spark.streaming.util.ManualClock
+
+import spark.{RDD, Logging}
+
+import collection.mutable.ArrayBuffer
+import collection.mutable.SynchronizedBuffer
+
+import java.io.{ObjectInputStream, IOException}
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+
+/**
+ * This is a input stream just for the testsuites. This is equivalent to a checkpointable,
+ * replayable, reliable message queue like Kafka. It requires a sequence as input, and
+ * returns the i_th element at the i_th batch unde manual clock.
+ */
+class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
+ extends InputDStream[T](ssc_) {
+
+ def start() {}
+
+ def stop() {}
+
+ def compute(validTime: Time): Option[RDD[T]] = {
+ logInfo("Computing RDD for time " + validTime)
+ val index = ((validTime - zeroTime) / slideDuration - 1).toInt
+ val selectedInput = if (index < input.size) input(index) else Seq[T]()
+ val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
+ logInfo("Created RDD " + rdd.id + " with " + selectedInput)
+ Some(rdd)
+ }
+}
+
+/**
+ * This is a output stream just for the testsuites. All the output is collected into a
+ * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ */
+class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
+ extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
+ val collected = rdd.collect()
+ output += collected
+ }) {
+
+ // This is to clear the output buffer every it is read from a checkpoint
+ @throws(classOf[IOException])
+ private def readObject(ois: ObjectInputStream) {
+ ois.defaultReadObject()
+ output.clear()
+ }
+}
+
+/**
+ * This is the base trait for Spark Streaming testsuites. This provides basic functionality
+ * to run user-defined set of input on user-defined stream operations, and verify the output.
+ */
+trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
+
+ def framework = "TestSuiteBase"
+
+ def master = "local[2]"
+
+ def batchDuration = Seconds(1)
+
+ def checkpointDir = "checkpoint"
+
+ def checkpointInterval = batchDuration
+
+ def numInputPartitions = 2
+
+ def maxWaitTimeMillis = 10000
+
+ def actuallyWait = false
+
+ /**
+ * Set up required DStreams to test the DStream operation using the two sequences
+ * of input collections.
+ */
+ def setupStreams[U: ClassManifest, V: ClassManifest](
+ input: Seq[Seq[U]],
+ operation: DStream[U] => DStream[V]
+ ): StreamingContext = {
+
+ // Create StreamingContext
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ if (checkpointDir != null) {
+ ssc.checkpoint(checkpointDir, checkpointInterval)
+ }
+
+ // Setup the stream computation
+ val inputStream = new TestInputStream(ssc, input, numInputPartitions)
+ val operatedStream = operation(inputStream)
+ val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[V]] with SynchronizedBuffer[Seq[V]])
+ ssc.registerInputStream(inputStream)
+ ssc.registerOutputStream(outputStream)
+ ssc
+ }
+
+ /**
+ * Set up required DStreams to test the binary operation using the sequence
+ * of input collections.
+ */
+ def setupStreams[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ input1: Seq[Seq[U]],
+ input2: Seq[Seq[V]],
+ operation: (DStream[U], DStream[V]) => DStream[W]
+ ): StreamingContext = {
+
+ // Create StreamingContext
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ if (checkpointDir != null) {
+ ssc.checkpoint(checkpointDir, checkpointInterval)
+ }
+
+ // Setup the stream computation
+ val inputStream1 = new TestInputStream(ssc, input1, numInputPartitions)
+ val inputStream2 = new TestInputStream(ssc, input2, numInputPartitions)
+ val operatedStream = operation(inputStream1, inputStream2)
+ val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[W]] with SynchronizedBuffer[Seq[W]])
+ ssc.registerInputStream(inputStream1)
+ ssc.registerInputStream(inputStream2)
+ ssc.registerOutputStream(outputStream)
+ ssc
+ }
+
+ /**
+ * Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
+ * returns the collected output. It will wait until `numExpectedOutput` number of
+ * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
+ */
+ def runStreams[V: ClassManifest](
+ ssc: StreamingContext,
+ numBatches: Int,
+ numExpectedOutput: Int
+ ): Seq[Seq[V]] = {
+
+ System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+ assert(numBatches > 0, "Number of batches to run stream computation is zero")
+ assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero")
+ logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput)
+
+ // Get the output buffer
+ val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
+ val output = outputStream.output
+
+ try {
+ // Start computation
+ ssc.start()
+
+ // Advance manual clock
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ logInfo("Manual clock before advancing = " + clock.time)
+ if (actuallyWait) {
+ for (i <- 1 to numBatches) {
+ logInfo("Actually waiting for " + batchDuration)
+ clock.addToTime(batchDuration.milliseconds)
+ Thread.sleep(batchDuration.milliseconds)
+ }
+ } else {
+ clock.addToTime(numBatches * batchDuration.milliseconds)
+ }
+ logInfo("Manual clock after advancing = " + clock.time)
+
+ // Wait until expected number of output items have been generated
+ val startTime = System.currentTimeMillis()
+ while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+ logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
+ Thread.sleep(100)
+ }
+ val timeTaken = System.currentTimeMillis() - startTime
+
+ assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+ assert(output.size === numExpectedOutput, "Unexpected number of outputs generated")
+
+ Thread.sleep(500) // Give some time for the forgetting old RDDs to complete
+ } catch {
+ case e: Exception => e.printStackTrace(); throw e;
+ } finally {
+ ssc.stop()
+ }
+
+ output
+ }
+
+ /**
+ * Verify whether the output values after running a DStream operation
+ * is same as the expected output values, by comparing the output
+ * collections either as lists (order matters) or sets (order does not matter)
+ */
+ def verifyOutput[V: ClassManifest](
+ output: Seq[Seq[V]],
+ expectedOutput: Seq[Seq[V]],
+ useSet: Boolean
+ ) {
+ logInfo("--------------------------------")
+ logInfo("output.size = " + output.size)
+ logInfo("output")
+ output.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("expected output.size = " + expectedOutput.size)
+ logInfo("expected output")
+ expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+ logInfo("--------------------------------")
+
+ // Match the output with the expected output
+ assert(output.size === expectedOutput.size, "Number of outputs do not match")
+ for (i <- 0 until output.size) {
+ if (useSet) {
+ assert(output(i).toSet === expectedOutput(i).toSet)
+ } else {
+ assert(output(i).toList === expectedOutput(i).toList)
+ }
+ }
+ logInfo("Output verified successfully")
+ }
+
+ /**
+ * Test unary DStream operation with a list of inputs, with number of
+ * batches to run same as the number of expected output values
+ */
+ def testOperation[U: ClassManifest, V: ClassManifest](
+ input: Seq[Seq[U]],
+ operation: DStream[U] => DStream[V],
+ expectedOutput: Seq[Seq[V]],
+ useSet: Boolean = false
+ ) {
+ testOperation[U, V](input, operation, expectedOutput, -1, useSet)
+ }
+
+ /**
+ * Test unary DStream operation with a list of inputs
+ * @param input Sequence of input collections
+ * @param operation Binary DStream operation to be applied to the 2 inputs
+ * @param expectedOutput Sequence of expected output collections
+ * @param numBatches Number of batches to run the operation for
+ * @param useSet Compare the output values with the expected output values
+ * as sets (order matters) or as lists (order does not matter)
+ */
+ def testOperation[U: ClassManifest, V: ClassManifest](
+ input: Seq[Seq[U]],
+ operation: DStream[U] => DStream[V],
+ expectedOutput: Seq[Seq[V]],
+ numBatches: Int,
+ useSet: Boolean
+ ) {
+ val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
+ val ssc = setupStreams[U, V](input, operation)
+ val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[V](output, expectedOutput, useSet)
+ }
+
+ /**
+ * Test binary DStream operation with two lists of inputs, with number of
+ * batches to run same as the number of expected output values
+ */
+ def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ input1: Seq[Seq[U]],
+ input2: Seq[Seq[V]],
+ operation: (DStream[U], DStream[V]) => DStream[W],
+ expectedOutput: Seq[Seq[W]],
+ useSet: Boolean
+ ) {
+ testOperation[U, V, W](input1, input2, operation, expectedOutput, -1, useSet)
+ }
+
+ /**
+ * Test binary DStream operation with two lists of inputs
+ * @param input1 First sequence of input collections
+ * @param input2 Second sequence of input collections
+ * @param operation Binary DStream operation to be applied to the 2 inputs
+ * @param expectedOutput Sequence of expected output collections
+ * @param numBatches Number of batches to run the operation for
+ * @param useSet Compare the output values with the expected output values
+ * as sets (order matters) or as lists (order does not matter)
+ */
+ def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ input1: Seq[Seq[U]],
+ input2: Seq[Seq[V]],
+ operation: (DStream[U], DStream[V]) => DStream[W],
+ expectedOutput: Seq[Seq[W]],
+ numBatches: Int,
+ useSet: Boolean
+ ) {
+ val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
+ val ssc = setupStreams[U, V, W](input1, input2, operation)
+ val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[W](output, expectedOutput, useSet)
+ }
+}
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
new file mode 100644
index 0000000000..0c6e928835
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -0,0 +1,300 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import collection.mutable.ArrayBuffer
+
+class WindowOperationsSuite extends TestSuiteBase {
+
+ override def framework = "WindowOperationsSuite"
+
+ override def maxWaitTimeMillis = 20000
+
+ override def batchDuration = Seconds(1)
+
+ after {
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ val largerSlideInput = Seq(
+ Seq(("a", 1)),
+ Seq(("a", 2)), // 1st window from here
+ Seq(("a", 3)),
+ Seq(("a", 4)), // 2nd window from here
+ Seq(("a", 5)),
+ Seq(("a", 6)), // 3rd window from here
+ Seq(),
+ Seq() // 4th window from here
+ )
+
+ val largerSlideReduceOutput = Seq(
+ Seq(("a", 3)),
+ Seq(("a", 10)),
+ Seq(("a", 18)),
+ Seq(("a", 11))
+ )
+
+
+ val bigInput = Seq(
+ Seq(("a", 1)),
+ Seq(("a", 1), ("b", 1)),
+ Seq(("a", 1), ("b", 1), ("c", 1)),
+ Seq(("a", 1), ("b", 1)),
+ Seq(("a", 1)),
+ Seq(),
+ Seq(("a", 1)),
+ Seq(("a", 1), ("b", 1)),
+ Seq(("a", 1), ("b", 1), ("c", 1)),
+ Seq(("a", 1), ("b", 1)),
+ Seq(("a", 1)),
+ Seq()
+ )
+
+ val bigGroupByOutput = Seq(
+ Seq(("a", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+ Seq(("a", Seq(1))),
+ Seq(("a", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+ Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+ Seq(("a", Seq(1)))
+ )
+
+
+ val bigReduceOutput = Seq(
+ Seq(("a", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 1)),
+ Seq(("a", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 1))
+ )
+
+ /*
+ The output of the reduceByKeyAndWindow with inverse reduce function is
+ different from the naive reduceByKeyAndWindow. Even if the count of a
+ particular key is 0, the key does not get eliminated from the RDDs of
+ ReducedWindowedDStream. This causes the number of keys in these RDDs to
+ increase forever. A more generalized version that allows elimination of
+ keys should be considered.
+ */
+
+ val bigReduceInvOutput = Seq(
+ Seq(("a", 1)),
+ Seq(("a", 2), ("b", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 1), ("c", 0)),
+ Seq(("a", 1), ("b", 0), ("c", 0)),
+ Seq(("a", 1), ("b", 0), ("c", 0)),
+ Seq(("a", 2), ("b", 1), ("c", 0)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 2), ("c", 1)),
+ Seq(("a", 2), ("b", 1), ("c", 0)),
+ Seq(("a", 1), ("b", 0), ("c", 0))
+ )
+
+ // Testing window operation
+
+ testWindow(
+ "basic window",
+ Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+ Seq( Seq(0), Seq(0, 1), Seq(1, 2), Seq(2, 3), Seq(3, 4), Seq(4, 5))
+ )
+
+ testWindow(
+ "tumbling window",
+ Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+ Seq( Seq(0, 1), Seq(2, 3), Seq(4, 5)),
+ Seconds(2),
+ Seconds(2)
+ )
+
+ testWindow(
+ "larger window",
+ Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+ Seq( Seq(0, 1), Seq(0, 1, 2, 3), Seq(2, 3, 4, 5), Seq(4, 5)),
+ Seconds(4),
+ Seconds(2)
+ )
+
+ testWindow(
+ "non-overlapping window",
+ Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+ Seq( Seq(1, 2), Seq(4, 5)),
+ Seconds(2),
+ Seconds(3)
+ )
+
+ // Testing naive reduceByKeyAndWindow (without invertible function)
+
+ testReduceByKeyAndWindow(
+ "basic reduction",
+ Seq( Seq(("a", 1), ("a", 3)) ),
+ Seq( Seq(("a", 4)) )
+ )
+
+ testReduceByKeyAndWindow(
+ "key already in window and new value added into window",
+ Seq( Seq(("a", 1)), Seq(("a", 1)) ),
+ Seq( Seq(("a", 1)), Seq(("a", 2)) )
+ )
+
+ testReduceByKeyAndWindow(
+ "new key added into window",
+ Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
+ Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
+ )
+
+ testReduceByKeyAndWindow(
+ "key removed from window",
+ Seq( Seq(("a", 1)), Seq(("a", 1)), Seq(), Seq() ),
+ Seq( Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 1)), Seq() )
+ )
+
+ testReduceByKeyAndWindow(
+ "larger slide time",
+ largerSlideInput,
+ largerSlideReduceOutput,
+ Seconds(4),
+ Seconds(2)
+ )
+
+ testReduceByKeyAndWindow("big test", bigInput, bigReduceOutput)
+
+ // Testing reduceByKeyAndWindow (with invertible reduce function)
+
+ testReduceByKeyAndWindowInv(
+ "basic reduction",
+ Seq(Seq(("a", 1), ("a", 3)) ),
+ Seq(Seq(("a", 4)) )
+ )
+
+ testReduceByKeyAndWindowInv(
+ "key already in window and new value added into window",
+ Seq( Seq(("a", 1)), Seq(("a", 1)) ),
+ Seq( Seq(("a", 1)), Seq(("a", 2)) )
+ )
+
+ testReduceByKeyAndWindowInv(
+ "new key added into window",
+ Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
+ Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
+ )
+
+ testReduceByKeyAndWindowInv(
+ "key removed from window",
+ Seq( Seq(("a", 1)), Seq(("a", 1)), Seq(), Seq() ),
+ Seq( Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 1)), Seq(("a", 0)) )
+ )
+
+ testReduceByKeyAndWindowInv(
+ "larger slide time",
+ largerSlideInput,
+ largerSlideReduceOutput,
+ Seconds(4),
+ Seconds(2)
+ )
+
+ testReduceByKeyAndWindowInv("big test", bigInput, bigReduceInvOutput)
+
+ test("groupByKeyAndWindow") {
+ val input = bigInput
+ val expectedOutput = bigGroupByOutput.map(_.map(x => (x._1, x._2.toSet)))
+ val windowDuration = Seconds(2)
+ val slideDuration = Seconds(1)
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[(String, Int)]) => {
+ s.groupByKeyAndWindow(windowDuration, slideDuration)
+ .map(x => (x._1, x._2.toSet))
+ .persist()
+ }
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+
+ test("countByWindow") {
+ val input = Seq(Seq(1), Seq(1), Seq(1, 2), Seq(0), Seq(), Seq() )
+ val expectedOutput = Seq( Seq(1), Seq(2), Seq(3), Seq(3), Seq(1), Seq(0))
+ val windowDuration = Seconds(2)
+ val slideDuration = Seconds(1)
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[Int]) => {
+ s.countByWindow(windowDuration, slideDuration).map(_.toInt)
+ }
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+
+ test("countByKeyAndWindow") {
+ val input = Seq(Seq(("a", 1)), Seq(("b", 1), ("b", 2)), Seq(("a", 10), ("b", 20)))
+ val expectedOutput = Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 2)), Seq(("a", 1), ("b", 3)))
+ val windowDuration = Seconds(2)
+ val slideDuration = Seconds(1)
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[(String, Int)]) => {
+ s.countByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt))
+ }
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+
+
+ // Helper functions
+
+ def testWindow(
+ name: String,
+ input: Seq[Seq[Int]],
+ expectedOutput: Seq[Seq[Int]],
+ windowDuration: Duration = Seconds(2),
+ slideDuration: Duration = Seconds(1)
+ ) {
+ test("window - " + name) {
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[Int]) => s.window(windowDuration, slideDuration)
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+ }
+
+ def testReduceByKeyAndWindow(
+ name: String,
+ input: Seq[Seq[(String, Int)]],
+ expectedOutput: Seq[Seq[(String, Int)]],
+ windowDuration: Duration = Seconds(2),
+ slideDuration: Duration = Seconds(1)
+ ) {
+ test("reduceByKeyAndWindow - " + name) {
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[(String, Int)]) => {
+ s.reduceByKeyAndWindow(_ + _, windowDuration, slideDuration).persist()
+ }
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+ }
+
+ def testReduceByKeyAndWindowInv(
+ name: String,
+ input: Seq[Seq[(String, Int)]],
+ expectedOutput: Seq[Seq[(String, Int)]],
+ windowDuration: Duration = Seconds(2),
+ slideDuration: Duration = Seconds(1)
+ ) {
+ test("reduceByKeyAndWindowInv - " + name) {
+ val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+ val operation = (s: DStream[(String, Int)]) => {
+ s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration)
+ .persist()
+ .checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing
+ }
+ testOperation(input, operation, expectedOutput, numBatches, true)
+ }
+ }
+}