Merge pull request #494 from mesos/streaming

Final set of Spark Streaming changes
This commit is contained in:
Matei Zaharia 2013-02-24 17:48:17 -08:00
commit 22f68b7e6b
66 changed files with 3291 additions and 1264 deletions

View file

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

View file

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

View file

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

View file

@ -183,7 +183,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.broadcast.factory</td>
<td>spark.broadcast. HttpBroadcastFactory</td>
<td>spark.broadcast.HttpBroadcastFactory</td>
<td>
Which broadcast implementation to use.
</td>
@ -244,10 +244,10 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
<td>spark.cleaner.delay</td>
<td>spark.cleaner.ttl</td>
<td>(disable)</td>
<td>
Duration (minutes) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is
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.

View file

@ -87,7 +87,7 @@ By default, the `pyspark` shell creates SparkContext that runs jobs locally.
To connect to a non-local cluster, set the `MASTER` environment variable.
For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
{% highlight shell %}
{% highlight bash %}
$ MASTER=spark://IP:PORT ./pyspark
{% endhighlight %}

View file

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

View file

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

View file

@ -20,11 +20,10 @@
<artifactId>jetty-server</artifactId>
</dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-stream</artifactId>
<version>3.0.3</version>
<groupId>com.twitter</groupId>
<artifactId>algebird-core_2.9.2</artifactId>
<version>0.1.8</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.version}</artifactId>

View file

@ -23,7 +23,7 @@ import spark.streaming.api.java.JavaStreamingContext;
*/
public class JavaNetworkWordCount {
public static void main(String[] args) {
if (args.length < 2) {
if (args.length < 3) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1");
System.exit(1);
@ -35,7 +35,7 @@ public class JavaNetworkWordCount {
// 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> lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2]));
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {

View file

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

View file

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

View file

@ -16,7 +16,7 @@ import spark.streaming.StreamingContext._
*/
object NetworkWordCount {
def main(args: Array[String]) {
if (args.length < 2) {
if (args.length < 3) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1")
System.exit(1)
@ -27,7 +27,7 @@ object NetworkWordCount {
// 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 lines = ssc.socketTextStream(args(1), args(2).toInt)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()

View file

@ -37,7 +37,7 @@ object RawNetworkGrep {
RawTextHelper.warmUp(ssc.sc)
val rawStreams = (1 to numStreams).map(_ =>
ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
val union = ssc.union(rawStreams)
union.filter(_.contains("the")).count().foreach(r =>
println("Grep count: " + r.collect().mkString))

View file

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

View file

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

View file

@ -1,19 +1,19 @@
package spark.streaming.examples.twitter
package spark.streaming.examples
import spark.streaming.StreamingContext._
import spark.streaming.{Seconds, StreamingContext}
import 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 {
object TwitterPopularTags {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: TwitterBasic <master> <twitter_username> <twitter_password>" +
System.err.println("Usage: TwitterPopularTags <master> <twitter_username> <twitter_password>" +
" [filter1] [filter2] ... [filter n]")
System.exit(1)
}
@ -21,10 +21,8 @@ object TwitterBasic {
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 ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2))
val stream = ssc.twitterStream(username, password, filters)
val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
@ -39,22 +37,17 @@ object TwitterBasic {
// 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))}
}
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))}
}
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()
}
}

View file

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

View file

@ -27,17 +27,16 @@ object PageViewStream {
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(_))
val pageViews = ssc.socketTextStream(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()
val pageCounts = pageViews.map(view => view.url).countByValue()
// 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()
val slidingPageCounts = pageViews.map(view => view.url)
.countByValueAndWindow(Seconds(10), Seconds(2))
// Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds

View file

@ -84,9 +84,9 @@
</snapshots>
</repository>
<repository>
<id>typesafe-repo</id>
<name>Typesafe Repository</name>
<url>http://repo.typesafe.com/typesafe/releases/</url>
<id>akka-repo</id>
<name>Akka Repository</name>
<url>http://repo.akka.io/releases/</url>
<releases>
<enabled>true</enabled>
</releases>

View file

@ -114,7 +114,7 @@ object SparkBuild extends Build {
def coreSettings = sharedSettings ++ Seq(
name := "spark-core",
resolvers ++= Seq(
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
"Akka Repository" at "http://repo.akka.io/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/",
@ -155,9 +155,7 @@ object SparkBuild extends Build {
def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples",
libraryDependencies ++= Seq(
"org.twitter4j" % "twitter4j-stream" % "3.0.3"
)
libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8")
)
def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
@ -166,7 +164,9 @@ object SparkBuild extends Build {
name := "spark-streaming",
libraryDependencies ++= Seq(
"org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
"com.github.sgroschupf" % "zkclient" % "0.1"
"com.github.sgroschupf" % "zkclient" % "0.1",
"org.twitter4j" % "twitter4j-stream" % "3.0.3",
"com.typesafe.akka" % "akka-zeromq" % "2.0.3"
)
) ++ assemblySettings ++ extraAssemblySettings

2
run
View file

@ -111,11 +111,13 @@ CLASSPATH+=":$FWDIR/conf"
CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes"
if [ -n "$SPARK_TESTING" ] ; then
CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
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"
CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
if [ -e "$FWDIR/lib_managed" ]; then
CLASSPATH+=":$FWDIR/lib_managed/jars/*"
CLASSPATH+=":$FWDIR/lib_managed/bundles/*"

View file

@ -47,11 +47,14 @@ set CORE_DIR=%FWDIR%core
set REPL_DIR=%FWDIR%repl
set EXAMPLES_DIR=%FWDIR%examples
set BAGEL_DIR=%FWDIR%bagel
set STREAMING_DIR=%FWDIR%streaming
set PYSPARK_DIR=%FWDIR%python
rem Build up classpath
set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources
set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes
set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\*
set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\*
set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\*

View file

@ -47,7 +47,16 @@
<artifactId>zkclient</artifactId>
<version>0.1</version>
</dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-stream</artifactId>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
<artifactId>akka-zeromq</artifactId>
<version>2.0.3</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.version}</artifactId>

View file

@ -6,6 +6,8 @@ import org.apache.hadoop.fs.{FileUtil, Path}
import org.apache.hadoop.conf.Configuration
import java.io._
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
import java.util.concurrent.Executors
private[streaming]
@ -17,7 +19,8 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
val jars = ssc.sc.jars
val graph = ssc.graph
val checkpointDir = ssc.checkpointDir
val checkpointDuration: Duration = ssc.checkpointDuration
val checkpointDuration = ssc.checkpointDuration
val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
def validate() {
assert(master != null, "Checkpoint.master is null")
@ -37,32 +40,50 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
val conf = new Configuration()
var fs = file.getFileSystem(conf)
val maxAttempts = 3
val executor = Executors.newFixedThreadPool(1)
class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
def run() {
var attempts = 0
val startTime = System.currentTimeMillis()
while (attempts < maxAttempts) {
attempts += 1
try {
logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
if (fs.exists(file)) {
val bkFile = new Path(file.getParent, file.getName + ".bk")
FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
logDebug("Moved existing checkpoint file to " + bkFile)
}
val fos = fs.create(file)
fos.write(bytes)
fos.close()
fos.close()
val finishTime = System.currentTimeMillis();
logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
"', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
return
} catch {
case ioe: IOException =>
logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
}
}
logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'")
}
}
def write(checkpoint: Checkpoint) {
// 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 + "'")
val bos = new ByteArrayOutputStream()
val zos = new LZFOutputStream(bos)
val oos = new ObjectOutputStream(zos)
oos.writeObject(checkpoint)
oos.close()
bos.close()
executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray))
}
def stop() {
executor.shutdown()
}
}
@ -84,7 +105,8 @@ object CheckpointReader extends Logging {
// 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 zis = new LZFInputStream(fis)
val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
val cp = ois.readObject.asInstanceOf[Checkpoint]
ois.close()
fs.close()

View file

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

View file

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

View file

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

View file

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

View file

@ -30,6 +30,7 @@ class Interval(val beginTime: Time, val endTime: Time) {
override def toString = "[" + beginTime + ", " + endTime + "]"
}
private[streaming]
object Interval {
def currentInterval(duration: Duration): Interval = {
val time = new Time(System.currentTimeMillis)

View file

@ -3,6 +3,8 @@ package spark.streaming
import spark.Logging
import spark.SparkEnv
import java.util.concurrent.Executors
import collection.mutable.HashMap
import collection.mutable.ArrayBuffer
private[streaming]
@ -13,21 +15,57 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging {
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))
logInfo("Total delay: %.5f s for job %s of time %s (execution: %.5f s)".format(
(System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, job.time.milliseconds, timeTaken / 1000.0))
} catch {
case e: Exception =>
logError("Running " + job + " failed", e)
}
clearJob(job)
}
}
initLogging()
val jobExecutor = Executors.newFixedThreadPool(numThreads)
val jobs = new HashMap[Time, ArrayBuffer[Job]]
def runJob(job: Job) {
jobs.synchronized {
jobs.getOrElseUpdate(job.time, new ArrayBuffer[Job]) += job
}
jobExecutor.execute(new JobHandler(ssc, job))
logInfo("Added " + job + " to queue")
}
def stop() {
jobExecutor.shutdown()
}
private def clearJob(job: Job) {
var timeCleared = false
val time = job.time
jobs.synchronized {
val jobsOfTime = jobs.get(time)
if (jobsOfTime.isDefined) {
jobsOfTime.get -= job
if (jobsOfTime.get.isEmpty) {
jobs -= time
timeCleared = true
}
} else {
throw new Exception("Job finished for time " + job.time +
" but time does not exist in jobs")
}
}
if (timeCleared) {
ssc.scheduler.clearOldMetadata(time)
}
}
def getPendingTimes(): Array[Time] = {
jobs.synchronized {
jobs.keySet.toArray
}
}
}

View file

@ -4,6 +4,7 @@ import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
import spark.Logging
import spark.SparkEnv
import spark.SparkContext._
import scala.collection.mutable.HashMap
import scala.collection.mutable.Queue
@ -23,7 +24,7 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) ext
*/
private[streaming]
class NetworkInputTracker(
@transient ssc: StreamingContext,
@transient ssc: StreamingContext,
@transient networkInputStreams: Array[NetworkInputDStream[_]])
extends Logging {
@ -65,12 +66,12 @@ class NetworkInputTracker(
def receive = {
case RegisterReceiver(streamId, receiverActor) => {
if (!networkInputStreamMap.contains(streamId)) {
throw new Exception("Register received for unexpected id " + 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)) {
@ -85,7 +86,7 @@ class NetworkInputTracker(
}
case DeregisterReceiver(streamId, msg) => {
receiverInfo -= streamId
logInfo("De-registered receiver for network stream " + streamId
logError("De-registered receiver for network stream " + streamId
+ " with message " + msg)
//TODO: Do something about the corresponding NetworkInputDStream
}
@ -95,8 +96,8 @@ class NetworkInputTracker(
/** This thread class runs all the receivers on the cluster. */
class ReceiverExecutor extends Thread {
val env = ssc.env
override def run() {
override def run() {
try {
SparkEnv.set(env)
startReceivers()
@ -113,7 +114,7 @@ class NetworkInputTracker(
*/
def startReceivers() {
val receivers = networkInputStreams.map(nis => {
val rcvr = nis.createReceiver()
val rcvr = nis.getReceiver()
rcvr.setStreamId(nis.id)
rcvr
})
@ -138,10 +139,14 @@ class NetworkInputTracker(
}
iterator.next().start()
}
// Run the dummy Spark job to ensure that all slaves have registered.
// This avoids all the receivers to be scheduled on the same node.
ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
// Distribute the receivers and start them
ssc.sc.runJob(tempRDD, startReceiver)
ssc.sparkContext.runJob(tempRDD, startReceiver)
}
/** Stops the receivers. */
def stopReceivers() {
// Signal the receivers to stop

View file

@ -18,15 +18,15 @@ 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 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
* Return 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])] = {
@ -34,7 +34,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
* Return 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])] = {
@ -42,7 +42,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]]
* Return 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])] = {
@ -54,7 +54,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -63,7 +63,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -72,7 +72,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -82,7 +82,7 @@ extends Serializable {
}
/**
* Combine elements of each key in DStream's RDDs using custom function. This is similar to the
* Combine elements of each key in DStream's RDDs using custom functions. This is similar to the
* combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
* information.
*/
@ -95,15 +95,7 @@ extends Serializable {
}
/**
* 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
* Return 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.
@ -115,7 +107,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `groupByKey` over a sliding window. Similar to
* Return 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
@ -129,7 +121,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
* Return 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
@ -137,7 +129,8 @@ extends Serializable {
* @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.
* @param numPartitions number of partitions of each RDD in the new DStream; if not specified
* then Spark's default number of partitions will be used
*/
def groupByKeyAndWindow(
windowDuration: Duration,
@ -155,7 +148,7 @@ extends Serializable {
* @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.
* @param partitioner partitioner for controlling the partitioning of each RDD in the new DStream.
*/
def groupByKeyAndWindow(
windowDuration: Duration,
@ -166,7 +159,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
* Return 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.
@ -182,7 +175,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
* Return 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
@ -201,7 +194,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
* Return 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
@ -210,10 +203,10 @@ extends Serializable {
* @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.
* @param numPartitions number of partitions of each RDD in the new DStream.
*/
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
reduceFunc: (V, V) => V,
windowDuration: Duration,
slideDuration: Duration,
numPartitions: Int
@ -222,7 +215,7 @@ extends Serializable {
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
* Return 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
@ -230,7 +223,8 @@ extends Serializable {
* @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.
* @param partitioner partitioner for controlling the partitioning of each RDD
* in the new DStream.
*/
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
@ -245,118 +239,78 @@ extends Serializable {
}
/**
* 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 :
* Return a new DStream by applying incremental `reduceByKey` over a sliding window.
* The reduced value of over a new window is calculated using the old window's reduced value :
* 1. reduce the new values that entered the window (e.g., adding new counts)
*
* 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
* This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
*
* This is more efficient than 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 invReduceFunc inverse reduce function
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
* @param filterFunc Optional function to filter expired key-value pairs;
* only pairs that satisfy the function are retained
*/
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
windowDuration: Duration,
slideDuration: Duration
slideDuration: Duration = self.slideDuration,
numPartitions: Int = ssc.sc.defaultParallelism,
filterFunc: ((K, V)) => Boolean = null
): 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
reduceFunc, invReduceFunc, windowDuration,
slideDuration, defaultPartitioner(numPartitions), filterFunc
)
}
/**
* Create a new "state" DStream where the state for each key is updated by applying
* Return a new DStream by applying incremental `reduceByKey` over a sliding window.
* The reduced value of over a new window is calculated using the old window's reduced value :
* 1. reduce the new values that entered the window (e.g., adding new counts)
* 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
* This is more efficient than reduceByKeyAndWindow without "inverse reduce" function.
* However, it is applicable to only "invertible reduce functions".
* @param reduceFunc associative reduce function
* @param invReduceFunc inverse reduce function
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
* @param partitioner partitioner for controlling the partitioning of each RDD in the new DStream.
* @param filterFunc Optional function to filter expired key-value pairs;
* only pairs that satisfy the function are retained
*/
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
windowDuration: Duration,
slideDuration: Duration,
partitioner: Partitioner,
filterFunc: ((K, V)) => Boolean
): DStream[(K, V)] = {
val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
val cleanedInvReduceFunc = ssc.sc.clean(invReduceFunc)
val cleanedFilterFunc = if (filterFunc != null) Some(ssc.sc.clean(filterFunc)) else None
new ReducedWindowedDStream[K, V](
self, cleanedReduceFunc, cleanedInvReduceFunc, cleanedFilterFunc,
windowDuration, slideDuration, partitioner
)
}
/**
* Return a new "state" DStream where the state for each key is updated by applying
* the given function on the previous state of the key and the new values of each key.
* 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
@ -370,7 +324,7 @@ extends Serializable {
}
/**
* Create a new "state" DStream where the state for each key is updated by applying
* Return a new "state" DStream where the state for each key is updated by applying
* the given function on the previous state of the key and the new values of each key.
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
* @param updateFunc State update function. If `this` function returns None, then
@ -405,7 +359,7 @@ extends Serializable {
}
/**
* Create a new "state" DStream where the state for each key is updated by applying
* Return a new "state" DStream where the state for each key is updated by applying
* the given function on the previous state of the key and the new values of each key.
* [[spark.Paxrtitioner]] is used to control the partitioning of each RDD.
* @param updateFunc State update function. If `this` function returns None, then
@ -447,7 +401,7 @@ extends Serializable {
}
/**
* Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
* Cogroup `this` DStream with `other` DStream using a partitioner. 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.
*/

View file

@ -9,11 +9,8 @@ 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 {
@ -23,54 +20,93 @@ class Scheduler(ssc: StreamingContext) extends Logging {
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)))
longTime => generateJobs(new Time(longTime)))
val graph = ssc.graph
var latestTime: Time = null
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.
def start() = synchronized {
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")
restart()
} else {
val firstTime = new Time(timer.start())
graph.start(firstTime - ssc.graph.batchDuration)
logInfo("Scheduler's timer started")
startFirstTime()
}
logInfo("Scheduler started")
}
def stop() {
def stop() = synchronized {
timer.stop()
graph.stop()
jobManager.stop()
if (checkpointWriter != null) checkpointWriter.stop()
ssc.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 startFirstTime() {
val startTime = new Time(timer.getStartTime())
graph.start(startTime - graph.batchDuration)
timer.start(startTime.milliseconds)
logInfo("Scheduler's timer started at " + startTime)
}
private def doCheckpoint(time: Time) {
private def restart() {
// 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.initialCheckpoint.checkpointTime.milliseconds
val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
}
val batchDuration = ssc.graph.batchDuration
// Batches when the master was down, that is,
// between the checkpoint and current restart time
val checkpointTime = ssc.initialCheckpoint.checkpointTime
val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds))
val downTimes = checkpointTime.until(restartTime, batchDuration)
logInfo("Batches during down time: " + downTimes.mkString(", "))
// Batches that were unprocessed before failure
val pendingTimes = ssc.initialCheckpoint.pendingTimes
logInfo("Batches pending processing: " + pendingTimes.mkString(", "))
// Reschedule jobs for these times
val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering)
logInfo("Batches to reschedule: " + timesToReschedule.mkString(", "))
timesToReschedule.foreach(time =>
graph.generateJobs(time).foreach(jobManager.runJob)
)
// Restart the timer
timer.start(restartTime.milliseconds)
logInfo("Scheduler's timer restarted at " + restartTime)
}
/** Generate jobs and perform checkpoint for the given `time`. */
def generateJobs(time: Time) {
SparkEnv.set(ssc.env)
logInfo("\n-----------------------------------------------------\n")
graph.generateJobs(time).foreach(jobManager.runJob)
latestTime = time
doCheckpoint(time)
}
/**
* Clear old metadata assuming jobs of `time` have finished processing.
* And also perform checkpoint.
*/
def clearOldMetadata(time: Time) {
ssc.graph.clearOldMetadata(time)
doCheckpoint(time)
}
/** Perform checkpoint for the give `time`. */
def doCheckpoint(time: Time) = synchronized {
if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
val startTime = System.currentTimeMillis()
logInfo("Checkpointing graph for time " + time)
ssc.graph.updateCheckpointData(time)
checkpointWriter.write(new Checkpoint(ssc, time))
val stopTime = System.currentTimeMillis()
logInfo("Checkpointing the graph took " + (stopTime - startTime) + " ms")
}
}
}

View file

@ -1,10 +1,19 @@
package spark.streaming
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.zeromq.Subscribe
import spark.streaming.dstream._
import spark.{RDD, Logging, SparkEnv, SparkContext}
import spark.streaming.receivers.ActorReceiver
import spark.streaming.receivers.ReceiverSupervisorStrategy
import spark.streaming.receivers.ZeroMQReceiver
import spark.storage.StorageLevel
import spark.util.MetadataCleaner
import spark.streaming.receivers.ActorReceiver
import scala.collection.mutable.Queue
@ -17,6 +26,7 @@ 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
import twitter4j.Status
/**
* A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@ -30,14 +40,14 @@ class StreamingContext private (
) extends Logging {
/**
* Creates a StreamingContext using an existing SparkContext.
* Create 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.
* Create 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 appName 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
@ -46,7 +56,7 @@ class StreamingContext private (
this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration)
/**
* Re-creates a StreamingContext from a checkpoint file.
* Re-create a StreamingContext from a checkpoint file.
* @param path Path either to the directory that was specified as the checkpoint directory, or
* to the checkpoint file 'graph' or 'graph.bk'.
*/
@ -61,7 +71,7 @@ class StreamingContext private (
protected[streaming] val isCheckpointPresent = (cp_ != null)
val sc: SparkContext = {
protected[streaming] val sc: SparkContext = {
if (isCheckpointPresent) {
new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars)
} else {
@ -101,7 +111,12 @@ class StreamingContext private (
protected[streaming] var scheduler: Scheduler = null
/**
* Sets each DStreams in this context to remember RDDs it generated in the last given duration.
* Return the associated Spark context
*/
def sparkContext = sc
/**
* Set 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).
@ -112,71 +127,119 @@ class StreamingContext private (
}
/**
* Sets the context to periodically checkpoint the DStream operations for master
* fault-tolerance. By default, the graph will be checkpointed every batch interval.
* Set the context to periodically checkpoint the DStream operations for master
* fault-tolerance. 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) {
def checkpoint(directory: String) {
if (directory != null) {
sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
checkpointDir = directory
checkpointDuration = interval
} else {
checkpointDir = null
checkpointDuration = null
}
}
protected[streaming] def getInitialCheckpoint(): Checkpoint = {
protected[streaming] def initialCheckpoint: Checkpoint = {
if (isCheckpointPresent) cp_ else null
}
protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement()
/**
* Create an input stream with any arbitrary user implemented network receiver.
* @param receiver Custom implementation of NetworkReceiver
*/
def networkStream[T: ClassManifest](
receiver: NetworkReceiver[T]): DStream[T] = {
val inputStream = new PluggableInputDStream[T](this,
receiver)
graph.addInputStream(inputStream)
inputStream
}
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
* @param name Name of the actor
* @param storageLevel RDD storage level. Defaults to memory-only.
*
* @note An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
def actorStream[T: ClassManifest](
props: Props,
name: String,
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = {
networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy))
}
/**
* Create an input stream that receives messages pushed by a zeromq publisher.
* @param publisherUrl Url of remote zeromq publisher
* @param subscribe topic to subscribe to
* @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
* of byte thus it needs the converter(which might be deserializer of bytes)
* to translate from sequence of sequence of bytes, where sequence refer to a frame
* and sub sequence refer to its payload.
* @param storageLevel RDD storage level. Defaults to memory-only.
*/
def zeroMQStream[T: ClassManifest](
publisherUrl:String,
subscribe: Subscribe,
bytesToObjects: Seq[Seq[Byte]] Iterator[T],
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
): DStream[T] = {
actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)),
"ZeroMQReceiver", storageLevel, supervisorStrategy)
}
/**
* Create an input stream that pulls messages form a Kafka Broker.
* @param hostname Zookeper hostname.
* @param port Zookeper port.
* @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
* @param 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.
* @param storageLevel Storage level to use for storing the received objects
* (default: StorageLevel.MEMORY_AND_DISK_SER_2)
*/
def kafkaStream[T: ClassManifest](
hostname: String,
port: Int,
zkQuorum: String,
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)
val inputStream = new KafkaInputDStream[T](this, zkQuorum, 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
* Create a input stream from TCP source hostname:port. Data is received using
* a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited
* 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(
def socketTextStream(
hostname: String,
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[String] = {
networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
}
/**
* Create a input stream from network source hostname:port. Data is received using
* Create a input stream from TCP 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
@ -185,7 +248,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects received (after converting bytes to objects)
*/
def networkStream[T: ClassManifest](
def socketStream[T: ClassManifest](
hostname: String,
port: Int,
converter: (InputStream) => Iterator[T],
@ -197,7 +260,7 @@ class StreamingContext private (
}
/**
* Creates a input stream from a Flume source.
* Create a input stream from a Flume source.
* @param hostname Hostname of the slave machine to which the flume data will be sent
* @param port Port of the slave machine to which the flume data will be sent
* @param storageLevel Storage level to use for storing the received objects
@ -222,7 +285,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects in the received blocks
*/
def rawNetworkStream[T: ClassManifest](
def rawSocketStream[T: ClassManifest](
hostname: String,
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
@ -233,7 +296,7 @@ class StreamingContext private (
}
/**
* Creates a input stream that monitors a Hadoop-compatible filesystem
* Create 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
@ -252,7 +315,7 @@ class StreamingContext private (
}
/**
* Creates a input stream that monitors a Hadoop-compatible filesystem
* Create 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
@ -271,9 +334,8 @@ class StreamingContext private (
inputStream
}
/**
* Creates a input stream that monitors a Hadoop-compatible filesystem
* Create 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
@ -283,17 +345,49 @@ class StreamingContext private (
}
/**
* Creates a input stream from an queue of RDDs. In each batch,
* Create a input stream that returns tweets received from Twitter.
* @param username Twitter username
* @param password Twitter password
* @param filters Set of filter strings to get only those tweets that match them
* @param storageLevel Storage level to use for storing the received objects
*/
def twitterStream(
username: String,
password: String,
filters: Seq[String] = Nil,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[Status] = {
val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel)
registerInputStream(inputStream)
inputStream
}
/**
* Create an input stream from a 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
oneAtATime: Boolean = true
): DStream[T] = {
queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1))
}
/**
* Create an input stream from a 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. Set as null if no RDD should be returned when empty
* @tparam T Type of objects in the RDD
*/
def queueStream[T: ClassManifest](
queue: Queue[RDD[T]],
oneAtATime: Boolean,
defaultRDD: RDD[T]
): DStream[T] = {
val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
registerInputStream(inputStream)
@ -308,7 +402,7 @@ class StreamingContext private (
}
/**
* Registers an input stream that will be started (InputDStream.start() called) to get the
* Register an input stream that will be started (InputDStream.start() called) to get the
* input data.
*/
def registerInputStream(inputStream: InputDStream[_]) {
@ -316,7 +410,7 @@ class StreamingContext private (
}
/**
* Registers an output stream that will be computed every interval
* Register an output stream that will be computed every interval
*/
def registerOutputStream(outputStream: DStream[_]) {
graph.addOutputStream(outputStream)
@ -334,7 +428,7 @@ class StreamingContext private (
}
/**
* Starts the execution of the streams.
* Start the execution of the streams.
*/
def start() {
if (checkpointDir != null && checkpointDuration == null && graph != null) {
@ -362,7 +456,7 @@ class StreamingContext private (
}
/**
* Sstops the execution of the streams.
* Stop the execution of the streams.
*/
def stop() {
try {
@ -408,4 +502,3 @@ object StreamingContext {
new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
}
}

View file

@ -37,6 +37,19 @@ case class Time(private val millis: Long) {
def max(that: Time): Time = if (this > that) this else that
def until(that: Time, interval: Duration): Seq[Time] = {
(this.milliseconds) until (that.milliseconds) by (interval.milliseconds) map (new Time(_))
}
def to(that: Time, interval: Duration): Seq[Time] = {
(this.milliseconds) to (that.milliseconds) by (interval.milliseconds) map (new Time(_))
}
override def toString: String = (millis.toString + " ms")
}
object Time {
val ordering = Ordering.by((time: Time) => time.millis)
}

View file

@ -4,6 +4,7 @@ import spark.streaming.{Duration, Time, DStream}
import spark.api.java.function.{Function => JFunction}
import spark.api.java.JavaRDD
import spark.storage.StorageLevel
import spark.RDD
/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
@ -16,9 +17,7 @@ import spark.storage.StorageLevel
*
* 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.
* only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`.
*
* DStreams internally is characterized by a few basic properties:
* - A list of other DStreams that the DStream depends on
@ -26,7 +25,9 @@ import spark.storage.StorageLevel
* - 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]] {
extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] {
override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
/** Return a new DStream containing only the elements that satisfy a predicate. */
def filter(f: JFunction[T, java.lang.Boolean]): JavaDStream[T] =
@ -36,7 +37,7 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
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()
def persist(): JavaDStream[T] = dstream.persist()
/** Persist the RDDs of this DStream with the given storage level */
def persist(storageLevel: StorageLevel): JavaDStream[T] = dstream.persist(storageLevel)
@ -50,33 +51,26 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
}
/**
* 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.
* Return a new DStream in which each RDD contains all the elements in seen in a
* sliding window of time over this DStream. The new DStream generates RDDs with
* the same interval as this DStream.
* @param windowDuration width of the window; must be a multiple of this DStream's interval.
* @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
* Return a new DStream in which each RDD contains all the elements in seen in a
* sliding window of time over this DStream.
* @param windowDuration width of the window; must be a multiple of this DStream's
* 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 interval
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching 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.

View file

@ -6,17 +6,20 @@ import java.lang.{Long => JLong}
import scala.collection.JavaConversions._
import spark.streaming._
import spark.api.java.JavaRDD
import spark.api.java.{JavaPairRDD, JavaRDDLike, 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 {
trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]
extends Serializable {
implicit val classManifest: ClassManifest[T]
def dstream: DStream[T]
def wrapRDD(in: RDD[T]): R
implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = {
in.map(new JLong(_))
}
@ -33,6 +36,26 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
*/
def count(): JavaDStream[JLong] = dstream.count()
/**
* Return a new DStream in which each RDD contains the counts of each distinct value in
* each RDD of this DStream. Hash partitioning is used to generate the RDDs with
* Spark's default number of partitions.
*/
def countByValue(): JavaPairDStream[T, JLong] = {
JavaPairDStream.scalaToJavaLong(dstream.countByValue())
}
/**
* Return a new DStream in which each RDD contains the counts of each distinct value in
* each RDD of this DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
* @param numPartitions number of partitions of each RDD in the new DStream.
*/
def countByValue(numPartitions: Int): JavaPairDStream[T, JLong] = {
JavaPairDStream.scalaToJavaLong(dstream.countByValue(numPartitions))
}
/**
* 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
@ -42,6 +65,39 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
dstream.countByWindow(windowDuration, slideDuration)
}
/**
* Return a new DStream in which each RDD contains the count of distinct elements in
* RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with
* 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 countByValueAndWindow(windowDuration: Duration, slideDuration: Duration)
: JavaPairDStream[T, JLong] = {
JavaPairDStream.scalaToJavaLong(
dstream.countByValueAndWindow(windowDuration, slideDuration))
}
/**
* Return a new DStream in which each RDD contains the count of distinct elements in
* RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
* @param numPartitions number of partitions of each RDD in the new DStream.
*/
def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
: JavaPairDStream[T, JLong] = {
JavaPairDStream.scalaToJavaLong(
dstream.countByValueAndWindow(windowDuration, slideDuration, numPartitions))
}
/**
* 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
@ -114,8 +170,38 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
/**
* 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)
* elements in a sliding window over this DStream.
* @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 reduceByWindow(
reduceFunc: (T, T) => T,
windowDuration: Duration,
slideDuration: Duration
): DStream[T] = {
dstream.reduceByWindow(reduceFunc, windowDuration, slideDuration)
}
/**
* Return a new DStream in which each RDD has a single element generated by reducing all
* elements in a sliding window over this DStream. However, the reduction is done incrementally
* using the old window's reduced value :
* 1. reduce the new values that entered the window (e.g., adding new counts)
* 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
* This is more efficient than reduceByWindow without "inverse reduce" function.
* However, it is applicable to only "invertible reduce functions".
* @param reduceFunc associative reduce function
* @param invReduceFunc inverse reduce function
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
* the new DStream will generate RDDs); must be a multiple of this
* DStream's batching interval
*/
def reduceByWindow(
reduceFunc: JFunction2[T, T, T],
@ -129,35 +215,35 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
/**
* 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)
def slice(fromTime: Time, toTime: Time): JList[R] = {
new util.ArrayList(dstream.slice(fromTime, toTime).map(wrapRDD(_)).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)))
def foreach(foreachFunc: JFunction[R, Void]) {
dstream.foreach(rdd => foreachFunc.call(wrapRDD(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))
def foreach(foreachFunc: JFunction2[R, Time, Void]) {
dstream.foreach((rdd, time) => foreachFunc.call(wrapRDD(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] = {
def transform[U](transformFunc: JFunction[R, 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
transformFunc.call(wrapRDD(in)).rdd
dstream.transform(scalaTransform(_))
}
@ -165,11 +251,41 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
* 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] = {
def transform[U](transformFunc: JFunction2[R, 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
transformFunc.call(wrapRDD(in), time).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[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
implicit val cmk: ClassManifest[K2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
implicit val cmv: ClassManifest[V2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
def scalaTransform (in: RDD[T]): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(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[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
implicit val cmk: ClassManifest[K2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
implicit val cmv: ClassManifest[V2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(in), time).rdd
dstream.transform(scalaTransform(_, _))
}

View file

@ -15,27 +15,30 @@ import org.apache.hadoop.conf.Configuration
import spark.api.java.{JavaRDD, JavaPairRDD}
import spark.storage.StorageLevel
import com.google.common.base.Optional
import spark.RDD
class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
implicit val kManifiest: ClassManifest[K],
implicit val vManifest: ClassManifest[V])
extends JavaDStreamLike[(K, V), JavaPairDStream[K, V]] {
extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
// =======================================================================
// Methods common to all DStream's
// =======================================================================
/** Returns a new DStream containing only the elements that satisfy a predicate. */
/** Return 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) */
/** Persist 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()
/** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
def persist(): JavaPairDStream[K, V] = dstream.persist()
/** Persists the RDDs of this DStream with the given storage level */
/** Persist 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 */
@ -67,70 +70,32 @@ class JavaPairDStream[K, V](val dstream: DStream[(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.
* 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: JavaPairDStream[K, V]): JavaPairDStream[K, V] =
dstream.union(that.dstream)
/**
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of this DStream.
*/
def transform[K2, V2](transformFunc: JFunction[JavaPairRDD[K, V], JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
implicit val cmk: ClassManifest[K2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
implicit val cmv: ClassManifest[V2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
def scalaTransform (in: RDD[(K, V)]): RDD[(K2, V2)] =
transformFunc.call(new JavaPairRDD[K, V](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[K2, V2](transformFunc: JFunction2[JavaPairRDD[K, V], Time, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
implicit val cmk: ClassManifest[K2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
implicit val cmv: ClassManifest[V2] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
def scalaTransform (in: RDD[(K, V)], time: Time): RDD[(K2, V2)] =
transformFunc.call(new JavaPairRDD[K, V](in), time).rdd
dstream.transform(scalaTransform(_, _))
}
// =======================================================================
// Methods only for PairDStream's
// =======================================================================
/**
* Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
* Return 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
* Return 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.
* Return 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.
@ -139,7 +104,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
dstream.groupByKey(partitioner).mapValues(seqAsJavaList _)
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -147,7 +112,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
dstream.reduceByKey(func)
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -155,7 +120,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
dstream.reduceByKey(func, numPartitions)
/**
* Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
* Return 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.
*/
@ -179,24 +144,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* 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
* Return 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.
@ -208,7 +156,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `groupByKey` over a sliding window. Similar to
* Return 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
@ -223,7 +171,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
* Return 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
@ -240,7 +188,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
* Return 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
@ -273,7 +221,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
* Return 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
@ -292,7 +240,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
* Return 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
@ -313,7 +261,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
* Return 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
@ -333,7 +281,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by reducing over a using incremental computation.
* Return 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)
@ -358,7 +306,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
* Create a new DStream by reducing over a using incremental computation.
* Return a new DStream by applying incremental `reduceByKey` over a sliding window.
* 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)
@ -372,25 +320,31 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* @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.
* @param numPartitions number of partitions of each RDD in the new DStream.
* @param filterFunc function to filter expired key-value pairs;
* only pairs that satisfy the function are retained
* set this to null if you do not want to filter
*/
def reduceByKeyAndWindow(
reduceFunc: Function2[V, V, V],
invReduceFunc: Function2[V, V, V],
windowDuration: Duration,
slideDuration: Duration,
numPartitions: Int
numPartitions: Int,
filterFunc: JFunction[(K, V), java.lang.Boolean]
): JavaPairDStream[K, V] = {
dstream.reduceByKeyAndWindow(
reduceFunc,
invReduceFunc,
windowDuration,
slideDuration,
numPartitions)
numPartitions,
(p: (K, V)) => filterFunc(p).booleanValue()
)
}
/**
* Create a new DStream by reducing over a using incremental computation.
* Return a new DStream by applying incremental `reduceByKey` over a sliding window.
* 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)
@ -404,49 +358,26 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* 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.
* @param filterFunc function to filter expired key-value pairs;
* only pairs that satisfy the function are retained
* set this to null if you do not want to filter
*/
def reduceByKeyAndWindow(
reduceFunc: Function2[V, V, V],
invReduceFunc: Function2[V, V, V],
windowDuration: Duration,
slideDuration: Duration,
partitioner: Partitioner
): JavaPairDStream[K, V] = {
partitioner: Partitioner,
filterFunc: JFunction[(K, V), java.lang.Boolean]
): 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)
partitioner,
(p: (K, V)) => filterFunc(p).booleanValue()
)
}
private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]):

View file

@ -1,16 +1,26 @@
package spark.streaming.api.java
import scala.collection.JavaConversions._
import java.lang.{Long => JLong, Integer => JInt}
import spark.streaming._
import dstream._
import receivers.{ActorReceiver, ReceiverSupervisorStrategy}
import spark.streaming.dstream._
import spark.storage.StorageLevel
import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
import spark.api.java.{JavaSparkContext, JavaRDD}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import twitter4j.Status
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.zeromq.Subscribe
import scala.collection.JavaConversions._
import java.lang.{Long => JLong, Integer => JInt}
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
@ -53,27 +63,24 @@ class JavaStreamingContext(val ssc: StreamingContext) {
/**
* Create an input stream that pulls messages form a Kafka Broker.
* @param hostname Zookeper hostname.
* @param port Zookeper port.
* @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
*/
def kafkaStream[T](
hostname: String,
port: Int,
zkQuorum: String,
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: _*))
ssc.kafkaStream[T](zkQuorum, 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 zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
@ -81,8 +88,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* By default the value is pulled from zookeper.
*/
def kafkaStream[T](
hostname: String,
port: Int,
zkQuorum: String,
groupId: String,
topics: JMap[String, JInt],
initialOffsets: JMap[KafkaPartitionKey, JLong])
@ -90,8 +96,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
implicit val cmt: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.kafkaStream[T](
hostname,
port,
zkQuorum,
groupId,
Map(topics.mapValues(_.intValue()).toSeq: _*),
Map(initialOffsets.mapValues(_.longValue()).toSeq: _*))
@ -99,8 +104,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
/**
* Create an input stream that pulls messages form a Kafka Broker.
* @param hostname Zookeper hostname.
* @param port Zookeper port.
* @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
@ -109,8 +113,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @param storageLevel RDD storage level. Defaults to memory-only
*/
def kafkaStream[T](
hostname: String,
port: Int,
zkQuorum: String,
groupId: String,
topics: JMap[String, JInt],
initialOffsets: JMap[KafkaPartitionKey, JLong],
@ -119,8 +122,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
implicit val cmt: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.kafkaStream[T](
hostname,
port,
zkQuorum,
groupId,
Map(topics.mapValues(_.intValue()).toSeq: _*),
Map(initialOffsets.mapValues(_.longValue()).toSeq: _*),
@ -136,9 +138,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @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)
def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel)
: JavaDStream[String] = {
ssc.networkTextStream(hostname, port, storageLevel)
ssc.socketTextStream(hostname, port, storageLevel)
}
/**
@ -148,8 +150,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @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)
def socketTextStream(hostname: String, port: Int): JavaDStream[String] = {
ssc.socketTextStream(hostname, port)
}
/**
@ -162,7 +164,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @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](
def socketStream[T](
hostname: String,
port: Int,
converter: JFunction[InputStream, java.lang.Iterable[T]],
@ -171,7 +173,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
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)
ssc.socketStream(hostname, port, fn, storageLevel)
}
/**
@ -194,13 +196,13 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects in the received blocks
*/
def rawNetworkStream[T](
def rawSocketStream[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))
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel))
}
/**
@ -212,10 +214,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @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] = {
def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = {
implicit val cmt: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port))
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port))
}
/**
@ -254,15 +256,182 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @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] = {
def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
ssc.flumeStream(hostname, port)
}
/**
* Create a input stream that returns tweets received from Twitter.
* @param username Twitter username
* @param password Twitter password
* @param filters Set of filter strings to get only those tweets that match them
* @param storageLevel Storage level to use for storing the received objects
*/
def twitterStream(
username: String,
password: String,
filters: Array[String],
storageLevel: StorageLevel
): JavaDStream[Status] = {
ssc.twitterStream(username, password, filters, storageLevel)
}
/**
* Create a input stream that returns tweets received from Twitter.
* @param username Twitter username
* @param password Twitter password
* @param filters Set of filter strings to get only those tweets that match them
*/
def twitterStream(
username: String,
password: String,
filters: Array[String]
): JavaDStream[Status] = {
ssc.twitterStream(username, password, filters)
}
/**
* Create a input stream that returns tweets received from Twitter.
* @param username Twitter username
* @param password Twitter password
*/
def twitterStream(
username: String,
password: String
): JavaDStream[Status] = {
ssc.twitterStream(username, password)
}
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
* @param name Name of the actor
* @param storageLevel Storage level to use for storing the received objects
*
* @note An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
def actorStream[T](
props: Props,
name: String,
storageLevel: StorageLevel,
supervisorStrategy: SupervisorStrategy
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.actorStream[T](props, name, storageLevel, supervisorStrategy)
}
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
* @param name Name of the actor
* @param storageLevel Storage level to use for storing the received objects
*
* @note An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
def actorStream[T](
props: Props,
name: String,
storageLevel: StorageLevel
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.actorStream[T](props, name, storageLevel)
}
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
* @param name Name of the actor
*
* @note An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
def actorStream[T](
props: Props,
name: String
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.actorStream[T](props, name)
}
/**
* Create an input stream that receives messages pushed by a zeromq publisher.
* @param publisherUrl Url of remote zeromq publisher
* @param subscribe topic to subscribe to
* @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
* of byte thus it needs the converter(which might be deserializer of bytes)
* to translate from sequence of sequence of bytes, where sequence refer to a frame
* and sub sequence refer to its payload.
* @param storageLevel Storage level to use for storing the received objects
*/
def zeroMQStream[T](
publisherUrl:String,
subscribe: Subscribe,
bytesToObjects: Seq[Seq[Byte]] Iterator[T],
storageLevel: StorageLevel,
supervisorStrategy: SupervisorStrategy
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
}
/**
* Create an input stream that receives messages pushed by a zeromq publisher.
* @param publisherUrl Url of remote zeromq publisher
* @param subscribe topic to subscribe to
* @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
* of byte thus it needs the converter(which might be deserializer of bytes)
* to translate from sequence of sequence of bytes, where sequence refer to a frame
* and sub sequence refer to its payload.
* @param storageLevel RDD storage level. Defaults to memory-only.
*/
def zeroMQStream[T](
publisherUrl:String,
subscribe: Subscribe,
bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
storageLevel: StorageLevel
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
}
/**
* Create an input stream that receives messages pushed by a zeromq publisher.
* @param publisherUrl Url of remote zeromq publisher
* @param subscribe topic to subscribe to
* @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
* of byte thus it needs the converter(which might be deserializer of bytes)
* to translate from sequence of sequence of bytes, where sequence refer to a frame
* and sub sequence refer to its payload.
*/
def zeroMQStream[T](
publisherUrl:String,
subscribe: Subscribe,
bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
): JavaDStream[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
}
/**
* Registers an output stream that will be computed every interval
*/
def registerOutputStream(outputStream: JavaDStreamLike[_, _]) {
def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {
ssc.registerOutputStream(outputStream.dstream)
}
@ -322,12 +491,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
/**
* Sets the context to periodically checkpoint the DStream operations for master
* fault-tolerance. By default, the graph will be checkpointed every batch interval.
* fault-tolerance. 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)
def checkpoint(directory: String) {
ssc.checkpoint(directory)
}
/**

View file

@ -2,13 +2,14 @@ package spark.streaming.dstream
import spark.RDD
import spark.rdd.UnionRDD
import spark.streaming.{StreamingContext, Time}
import spark.streaming.{DStreamCheckpointData, 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
import scala.collection.mutable.{HashSet, HashMap}
import java.io.{ObjectInputStream, IOException}
private[streaming]
class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
@ -18,28 +19,23 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
newFilesOnly: Boolean = true)
extends InputDStream[(K, V)](ssc_) {
protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
// Latest file mod time seen till any point of time
private val lastModTimeFiles = new HashSet[String]()
private var lastModTime = 0L
@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_
}
@transient private[streaming] var files = new HashMap[Time, Array[String]]
override def start() {
if (newFilesOnly) {
lastModTime = System.currentTimeMillis()
lastModTime = graph.zeroTime.milliseconds
} else {
lastModTime = 0
}
logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
}
override def stop() { }
@ -49,38 +45,50 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
* 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.
* granularity of seconds. And new files may have the same modification time as the
* latest modification time in the previous call to this method yet was not reported in
* the previous call.
*/
override def compute(validTime: Time): Option[RDD[(K, V)]] = {
assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime)
// Create the filter for selecting new files
val newFilter = new PathFilter() {
// Latest file mod time seen in this round of fetching files and its corresponding files
var latestModTime = 0L
val latestModTimeFiles = new HashSet[String]()
def accept(path: Path): Boolean = {
if (!filter(path)) {
if (!filter(path)) { // Reject file if it does not satisfy filter
logDebug("Rejected by filter " + path)
return false
} else {
} else { // Accept file only if
val modTime = fs.getFileStatus(path).getModificationTime()
if (modTime < lastModTime){
return false
logDebug("Mod time for " + path + " is " + modTime)
if (modTime < lastModTime) {
logDebug("Mod time less than last mod time")
return false // If the file was created before the last time it was called
} else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) {
return false
logDebug("Mod time equal to last mod time, but file considered already")
return false // If the file was created exactly as lastModTime but not reported yet
} else if (modTime > validTime.milliseconds) {
logDebug("Mod time more than valid time")
return false // If the file was created after the time this function call requires
}
if (modTime > latestModTime) {
latestModTime = modTime
latestModTimeFiles.clear()
logDebug("Latest mod time updated to " + latestModTime)
}
latestModTimeFiles += path.toString
logDebug("Accepted " + path)
return true
}
}
}
val newFiles = fs.listStatus(path, newFilter)
logInfo("New files: " + newFiles.map(_.getPath).mkString(", "))
logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime)
val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString)
logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n"))
if (newFiles.length > 0) {
// Update the modification time and the files processed for that modification time
if (lastModTime != newFilter.latestModTime) {
@ -88,10 +96,81 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
lastModTimeFiles.clear()
}
lastModTimeFiles ++= newFilter.latestModTimeFiles
logDebug("Last mod time updated to " + lastModTime)
}
files += ((validTime, newFiles))
Some(filesToRDD(newFiles))
}
/** Clear the old time-to-files mappings along with old RDDs */
protected[streaming] override def clearOldMetadata(time: Time) {
super.clearOldMetadata(time)
val oldFiles = files.filter(_._1 <= (time - rememberDuration))
files --= oldFiles.keys
logInfo("Cleared " + oldFiles.size + " old files that were older than " +
(time - rememberDuration) + ": " + oldFiles.keys.mkString(", "))
logDebug("Cleared files are:\n" +
oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n"))
}
/** Generate one RDD from an array of files */
protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
new UnionRDD(
context.sparkContext,
files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
)
}
private def path: Path = {
if (path_ == null) path_ = new Path(directory)
path_
}
private def fs: FileSystem = {
if (fs_ == null) fs_ = path.getFileSystem(new Configuration())
fs_
}
@throws(classOf[IOException])
private def readObject(ois: ObjectInputStream) {
logDebug(this.getClass().getSimpleName + ".readObject used")
ois.defaultReadObject()
generatedRDDs = new HashMap[Time, RDD[(K,V)]] ()
files = new HashMap[Time, Array[String]]
}
/**
* A custom version of the DStreamCheckpointData that stores names of
* Hadoop files as checkpoint data.
*/
private[streaming]
class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) {
def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]]
override def update() {
hadoopFiles.clear()
hadoopFiles ++= files
}
override def cleanup() { }
override def restore() {
hadoopFiles.foreach {
case (t, f) => {
// Restore the metadata in both files and generatedRDDs
logInfo("Restoring files for time " + t + " - " +
f.mkString("[", ", ", "]") )
files += ((t, f))
generatedRDDs += ((t, filesToRDD(f)))
}
}
}
override def toString() = {
"[\n" + hadoopFiles.size + " file sets\n" +
hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]"
}
val newRDD = new UnionRDD(ssc.sc, newFiles.map(
file => ssc.sc.newAPIHadoopFile[K, V, F](file.getPath.toString)))
Some(newRDD)
}
}
@ -100,3 +179,4 @@ object FileInputDStream {
def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
}

View file

@ -25,7 +25,7 @@ class FlumeInputDStream[T: ClassManifest](
storageLevel: StorageLevel
) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
override def createReceiver(): NetworkReceiver[SparkFlumeEvent] = {
override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = {
new FlumeReceiver(host, port, storageLevel)
}
}
@ -134,4 +134,4 @@ class FlumeReceiver(
}
override def getLocationPreference = Some(host)
}
}

View file

@ -1,10 +1,42 @@
package spark.streaming.dstream
import spark.streaming.{Duration, StreamingContext, DStream}
import spark.streaming.{Time, Duration, StreamingContext, DStream}
/**
* This is the abstract base class for all input streams. This class provides to methods
* start() and stop() which called by the scheduler to start and stop receiving data/
* Input streams that can generated RDDs from new data just by running a service on
* the driver node (that is, without running a receiver onworker nodes) can be
* implemented by directly subclassing this InputDStream. For example,
* FileInputDStream, a subclass of InputDStream, monitors a HDFS directory for
* new files and generates RDDs on the new files. For implementing input streams
* that requires running a receiver on the worker nodes, use NetworkInputDStream
* as the parent class.
*/
abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
extends DStream[T](ssc_) {
var lastValidTime: Time = null
/**
* Checks whether the 'time' is valid wrt slideDuration for generating RDD.
* Additionally it also ensures valid times are in strictly increasing order.
* This ensures that InputDStream.compute() is called strictly on increasing
* times.
*/
override protected def isTimeValid(time: Time): Boolean = {
if (!super.isTimeValid(time)) {
false // Time not valid
} else {
// Time is valid, but check it it is more than lastValidTime
if (lastValidTime != null && time < lastValidTime) {
logWarning("isTimeValid called with " + time + " where as last valid time is " + lastValidTime)
}
lastValidTime = time
true
}
}
override def dependencies = List()
override def slideDuration: Duration = {
@ -13,7 +45,9 @@ abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContex
ssc.graph.batchDuration
}
/** Method called to start receiving data. Subclasses must implement this method. */
def start()
/** Method called to stop receiving data. Subclasses must implement this method. */
def stop()
}

View file

@ -19,21 +19,11 @@ 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 zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
* @param groupId The group id for this consumer.
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
@ -44,65 +34,22 @@ case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any],
private[streaming]
class KafkaInputDStream[T: ClassManifest](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
zkQuorum: String,
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)
def getReceiver(): NetworkReceiver[T] = {
new KafkaReceiver(zkQuorum, groupId, topics, initialOffsets, storageLevel)
.asInstanceOf[NetworkReceiver[T]]
}
}
private[streaming]
class KafkaReceiver(host: String, port: Int, groupId: String,
class KafkaReceiver(zkQuorum: String, groupId: String,
topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long],
storageLevel: StorageLevel) extends NetworkReceiver[Any] {
@ -111,8 +58,6 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
// 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
@ -127,24 +72,23 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
// 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.connect", zkQuorum)
props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString)
props.put("groupid", groupId)
// Create the connection to the cluster
logInfo("Connecting to Zookeper: " + zooKeeperEndPoint)
logInfo("Connecting to Zookeper: " + zkQuorum)
val consumerConfig = new ConsumerConfig(props)
consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector]
logInfo("Connected to " + zooKeeperEndPoint)
logInfo("Connected to " + zkQuorum)
// Reset the Kafka offsets in case we are recovering from a failure
resetOffsets(initialOffsets)
// If specified, set the topic offset
setOffsets(initialOffsets)
// Create Threads for each Topic/Message Stream we are listening
val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder())
@ -157,11 +101,11 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
}
// Overwrites the offets in Zookeper.
private def resetOffsets(offsets: Map[KafkaPartitionKey, Long]) {
private def setOffsets(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,
updatePersistentPath(consumerConnector.zkClient,
topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString)
}
}
@ -172,29 +116,10 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
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))
// }
// }
}

View file

@ -20,7 +20,7 @@ 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
* define the getReceiver() function that gets 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
@ -34,11 +34,11 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
val id = ssc.getNewNetworkStreamId()
/**
* Creates the receiver object that will be sent to the worker nodes
* Gets 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]
def getReceiver(): NetworkReceiver[T]
// Nothing to start or stop as both taken care of by the NetworkInputTracker.
def start() {}
@ -46,8 +46,15 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
def stop() {}
override def compute(validTime: Time): Option[RDD[T]] = {
val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
Some(new BlockRDD[T](ssc.sc, blockIds))
// If this is called for any time before the start time of the context,
// then this returns an empty RDD. This may happen when recovering from a
// master failure
if (validTime >= graph.startTime) {
val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
Some(new BlockRDD[T](ssc.sc, blockIds))
} else {
Some(new BlockRDD[T](ssc.sc, Array[String]()))
}
}
}

View file

@ -0,0 +1,13 @@
package spark.streaming.dstream
import spark.streaming.StreamingContext
private[streaming]
class PluggableInputDStream[T: ClassManifest](
@transient ssc_ : StreamingContext,
receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) {
def getReceiver(): NetworkReceiver[T] = {
receiver
}
}

View file

@ -7,6 +7,7 @@ import scala.collection.mutable.Queue
import scala.collection.mutable.ArrayBuffer
import spark.streaming.{Time, StreamingContext}
private[streaming]
class QueueInputDStream[T: ClassManifest](
@transient ssc: StreamingContext,
val queue: Queue[RDD[T]],

View file

@ -25,7 +25,7 @@ class RawInputDStream[T: ClassManifest](
storageLevel: StorageLevel
) extends NetworkInputDStream[T](ssc_ ) with Logging {
def createReceiver(): NetworkReceiver[T] = {
def getReceiver(): NetworkReceiver[T] = {
new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]]
}
}

View file

@ -3,7 +3,7 @@ package spark.streaming.dstream
import spark.streaming.StreamingContext._
import spark.RDD
import spark.rdd.CoGroupedRDD
import spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
import spark.Partitioner
import spark.SparkContext._
import spark.storage.StorageLevel
@ -15,7 +15,8 @@ private[streaming]
class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
parent: DStream[(K, V)],
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
filterFunc: Option[((K, V)) => Boolean],
_windowDuration: Duration,
_slideDuration: Duration,
partitioner: Partitioner
@ -87,15 +88,18 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
//
// Get the RDDs of the reduced values in "old time steps"
val oldRDDs = reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration)
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)
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)]()))
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
@ -114,7 +118,9 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
// 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)
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) {
@ -140,10 +146,12 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues)
Some(mergedValuesRDD)
if (filterFunc.isDefined) {
Some(mergedValuesRDD.filter(filterFunc.get))
} else {
Some(mergedValuesRDD)
}
}
}

View file

@ -15,7 +15,7 @@ class SocketInputDStream[T: ClassManifest](
storageLevel: StorageLevel
) extends NetworkInputDStream[T](ssc_) {
def createReceiver(): NetworkReceiver[T] = {
def getReceiver(): NetworkReceiver[T] = {
new SocketReceiver(host, port, bytesToObjects, storageLevel)
}
}

View file

@ -48,8 +48,16 @@ class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
//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 parent RDD does not exist
// Re-apply the update function to the old state RDD
val updateFuncLocal = updateFunc
val finalFunc = (iterator: Iterator[(K, S)]) => {
val i = iterator.map(t => (t._1, Seq[V](), Option(t._2)))
updateFuncLocal(i)
}
val stateRDD = prevStateRDD.mapPartitions(finalFunc, preservePartitioning)
return Some(stateRDD)
}
}
}

View file

@ -1,12 +1,11 @@
package spark.streaming.examples.twitter
package spark.streaming.dstream
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.
*
@ -14,19 +13,21 @@ import collection.JavaConversions._
* 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.
*/
private[streaming]
class TwitterInputDStream(
@transient ssc_ : StreamingContext,
username: String,
password: String,
filters: Seq[String],
storageLevel: StorageLevel
) extends NetworkInputDStream[Status](ssc_) {
) extends NetworkInputDStream[Status](ssc_) {
override def createReceiver(): NetworkReceiver[Status] = {
override def getReceiver(): NetworkReceiver[Status] = {
new TwitterReceiver(username, password, filters, storageLevel)
}
}
private[streaming]
class TwitterReceiver(
username: String,
password: String,
@ -50,7 +51,7 @@ class TwitterReceiver(
def onTrackLimitationNotice(i: Int) {}
def onScrubGeo(l: Long, l1: Long) {}
def onStallWarning(stallWarning: StallWarning) {}
def onException(e: Exception) {}
def onException(e: Exception) { stopOnError(e) }
})
val query: FilterQuery = new FilterQuery

View file

@ -0,0 +1,153 @@
package spark.streaming.receivers
import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
import akka.actor.{ actorRef2Scala, ActorRef }
import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
import spark.storage.StorageLevel
import spark.streaming.dstream.NetworkReceiver
import java.util.concurrent.atomic.AtomicInteger
/** A helper with set of defaults for supervisor strategy **/
object ReceiverSupervisorStrategy {
import akka.util.duration._
import akka.actor.SupervisorStrategy._
val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange =
15 millis) {
case _: RuntimeException Restart
case _: Exception Escalate
}
}
/**
* A receiver trait to be mixed in with your Actor to gain access to
* pushBlock API.
*
* @example {{{
* class MyActor extends Actor with Receiver{
* def receive {
* case anything :String pushBlock(anything)
* }
* }
* //Can be plugged in actorStream as follows
* ssc.actorStream[String](Props(new MyActor),"MyActorReceiver")
*
* }}}
*
* @note An important point to note:
* Since Actor may exist outside the spark framework, It is thus user's responsibility
* to ensure the type safety, i.e parametrized type of push block and InputDStream
* should be same.
*
*/
trait Receiver { self: Actor
def pushBlock[T: ClassManifest](iter: Iterator[T]) {
context.parent ! Data(iter)
}
def pushBlock[T: ClassManifest](data: T) {
context.parent ! Data(data)
}
}
/**
* Statistics for querying the supervisor about state of workers
*/
case class Statistics(numberOfMsgs: Int,
numberOfWorkers: Int,
numberOfHiccups: Int,
otherInfo: String)
/** Case class to receive data sent by child actors **/
private[streaming] case class Data[T: ClassManifest](data: T)
/**
* Provides Actors as receivers for receiving stream.
*
* As Actors can also be used to receive data from almost any stream source.
* A nice set of abstraction(s) for actors as receivers is already provided for
* a few general cases. It is thus exposed as an API where user may come with
* his own Actor to run as receiver for Spark Streaming input source.
*
* This starts a supervisor actor which starts workers and also provides
* [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
*
* Here's a way to start more supervisor/workers as its children.
*
* @example {{{
* context.parent ! Props(new Supervisor)
* }}} OR {{{
* context.parent ! Props(new Worker,"Worker")
* }}}
*
*
*/
private[streaming] class ActorReceiver[T: ClassManifest](
props: Props,
name: String,
storageLevel: StorageLevel,
receiverSupervisorStrategy: SupervisorStrategy)
extends NetworkReceiver[T] {
protected lazy val blocksGenerator: BlockGenerator =
new BlockGenerator(storageLevel)
protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor),
"Supervisor" + streamId)
private class Supervisor extends Actor {
override val supervisorStrategy = receiverSupervisorStrategy
val worker = context.actorOf(props, name)
logInfo("Started receiver worker at:" + worker.path)
val n: AtomicInteger = new AtomicInteger(0)
val hiccups: AtomicInteger = new AtomicInteger(0)
def receive = {
case Data(iter: Iterator[_]) pushBlock(iter.asInstanceOf[Iterator[T]])
case Data(msg)
blocksGenerator += msg.asInstanceOf[T]
n.incrementAndGet
case props: Props
val worker = context.actorOf(props)
logInfo("Started receiver worker at:" + worker.path)
sender ! worker
case (props: Props, name: String)
val worker = context.actorOf(props, name)
logInfo("Started receiver worker at:" + worker.path)
sender ! worker
case _: PossiblyHarmful => hiccups.incrementAndGet()
case _: Statistics
val workers = context.children
sender ! Statistics(n.get, workers.size, hiccups.get, workers.mkString("\n"))
}
}
protected def pushBlock(iter: Iterator[T]) {
pushBlock("block-" + streamId + "-" + System.nanoTime(),
iter, null, storageLevel)
}
protected def onStart() = {
blocksGenerator.start()
supervisor
logInfo("Supervision tree for receivers initialized at:" + supervisor.path)
}
protected def onStop() = {
supervisor ! PoisonPill
}
}

View file

@ -0,0 +1,33 @@
package spark.streaming.receivers
import akka.actor.Actor
import akka.zeromq._
import spark.Logging
/**
* A receiver to subscribe to ZeroMQ stream.
*/
private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
subscribe: Subscribe,
bytesToObjects: Seq[Seq[Byte]] Iterator[T])
extends Actor with Receiver with Logging {
override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self),
Connect(publisherUrl), subscribe)
def receive: Receive = {
case Connecting logInfo("connecting ...")
case m: ZMQMessage
logDebug("Received message for:" + m.firstFrameAsString)
//We ignore first frame for processing as it is the topic
val bytes = m.frames.tail.map(_.payload)
pushBlock(bytesToObjects(bytes))
case Closed logInfo("received closed ")
}
}

View file

@ -0,0 +1,392 @@
package spark.streaming.util
import spark.{Logging, RDD}
import spark.streaming._
import spark.streaming.dstream.ForEachDStream
import StreamingContext._
import scala.util.Random
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
import java.io.{File, ObjectInputStream, IOException}
import java.util.UUID
import com.google.common.io.Files
import org.apache.commons.io.FileUtils
import org.apache.hadoop.fs.{FileUtil, FileSystem, Path}
import org.apache.hadoop.conf.Configuration
private[streaming]
object MasterFailureTest extends Logging {
initLogging()
@volatile var killed = false
@volatile var killCount = 0
def main(args: Array[String]) {
if (args.size < 2) {
println(
"Usage: MasterFailureTest <local/HDFS directory> <# batches> [<batch size in milliseconds>]")
System.exit(1)
}
val directory = args(0)
val numBatches = args(1).toInt
val batchDuration = if (args.size > 2) Milliseconds(args(2).toInt) else Seconds(1)
println("\n\n========================= MAP TEST =========================\n\n")
testMap(directory, numBatches, batchDuration)
println("\n\n================= UPDATE-STATE-BY-KEY TEST =================\n\n")
testUpdateStateByKey(directory, numBatches, batchDuration)
println("\n\nSUCCESS\n\n")
}
def testMap(directory: String, numBatches: Int, batchDuration: Duration) {
// Input: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ...
val input = (1 to numBatches).map(_.toString).toSeq
// Expected output: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ...
val expectedOutput = (1 to numBatches)
val operation = (st: DStream[String]) => st.map(_.toInt)
// Run streaming operation with multiple master failures
val output = testOperation(directory, batchDuration, input, operation, expectedOutput)
logInfo("Expected output, size = " + expectedOutput.size)
logInfo(expectedOutput.mkString("[", ",", "]"))
logInfo("Output, size = " + output.size)
logInfo(output.mkString("[", ",", "]"))
// Verify whether all the values of the expected output is present
// in the output
assert(output.distinct.toSet == expectedOutput.toSet)
}
def testUpdateStateByKey(directory: String, numBatches: Int, batchDuration: Duration) {
// Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
val input = (1 to numBatches).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq
// Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ...
val expectedOutput = (1L to numBatches).map(i => (1L to i).reduce(_ + _)).map(j => ("a", j))
val operation = (st: DStream[String]) => {
val updateFunc = (values: Seq[Long], state: Option[Long]) => {
Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L))
}
st.flatMap(_.split(" "))
.map(x => (x, 1L))
.updateStateByKey[Long](updateFunc)
.checkpoint(batchDuration * 5)
}
// Run streaming operation with multiple master failures
val output = testOperation(directory, batchDuration, input, operation, expectedOutput)
logInfo("Expected output, size = " + expectedOutput.size + "\n" + expectedOutput)
logInfo("Output, size = " + output.size + "\n" + output)
// Verify whether all the values in the output are among the expected output values
output.foreach(o =>
assert(expectedOutput.contains(o), "Expected value " + o + " not found")
)
// Verify whether the last expected output value has been generated, there by
// confirming that none of the inputs have been missed
assert(output.last == expectedOutput.last)
}
/**
* Tests stream operation with multiple master failures, and verifies whether the
* final set of output values is as expected or not.
*/
def testOperation[T: ClassManifest](
directory: String,
batchDuration: Duration,
input: Seq[String],
operation: DStream[String] => DStream[T],
expectedOutput: Seq[T]
): Seq[T] = {
// Just making sure that the expected output does not have duplicates
assert(expectedOutput.distinct.toSet == expectedOutput.toSet)
// Setup the stream computation with the given operation
val (ssc, checkpointDir, testDir) = setupStreams(directory, batchDuration, operation)
// Start generating files in the a different thread
val fileGeneratingThread = new FileGeneratingThread(input, testDir, batchDuration.milliseconds)
fileGeneratingThread.start()
// Run the streams and repeatedly kill it until the last expected output
// has been generated, or until it has run for twice the expected time
val lastExpectedOutput = expectedOutput.last
val maxTimeToRun = expectedOutput.size * batchDuration.milliseconds * 2
val mergedOutput = runStreams(ssc, lastExpectedOutput, maxTimeToRun)
// Delete directories
fileGeneratingThread.join()
val fs = checkpointDir.getFileSystem(new Configuration())
fs.delete(checkpointDir, true)
fs.delete(testDir, true)
logInfo("Finished test after " + killCount + " failures")
mergedOutput
}
/**
* Sets up the stream computation with the given operation, directory (local or HDFS),
* and batch duration. Returns the streaming context and the directory to which
* files should be written for testing.
*/
private def setupStreams[T: ClassManifest](
directory: String,
batchDuration: Duration,
operation: DStream[String] => DStream[T]
): (StreamingContext, Path, Path) = {
// Reset all state
reset()
// Create the directories for this test
val uuid = UUID.randomUUID().toString
val rootDir = new Path(directory, uuid)
val fs = rootDir.getFileSystem(new Configuration())
val checkpointDir = new Path(rootDir, "checkpoint")
val testDir = new Path(rootDir, "test")
fs.mkdirs(checkpointDir)
fs.mkdirs(testDir)
// Setup the streaming computation with the given operation
System.clearProperty("spark.driver.port")
var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration)
ssc.checkpoint(checkpointDir.toString)
val inputStream = ssc.textFileStream(testDir.toString)
val operatedStream = operation(inputStream)
val outputStream = new TestOutputStream(operatedStream)
ssc.registerOutputStream(outputStream)
(ssc, checkpointDir, testDir)
}
/**
* Repeatedly starts and kills the streaming context until timed out or
* the last expected output is generated. Finally, return
*/
private def runStreams[T: ClassManifest](
ssc_ : StreamingContext,
lastExpectedOutput: T,
maxTimeToRun: Long
): Seq[T] = {
var ssc = ssc_
var totalTimeRan = 0L
var isLastOutputGenerated = false
var isTimedOut = false
val mergedOutput = new ArrayBuffer[T]()
val checkpointDir = ssc.checkpointDir
var batchDuration = ssc.graph.batchDuration
while(!isLastOutputGenerated && !isTimedOut) {
// Get the output buffer
val outputBuffer = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[T]].output
def output = outputBuffer.flatMap(x => x)
// Start the thread to kill the streaming after some time
killed = false
val killingThread = new KillingThread(ssc, batchDuration.milliseconds * 10)
killingThread.start()
var timeRan = 0L
try {
// Start the streaming computation and let it run while ...
// (i) StreamingContext has not been shut down yet
// (ii) The last expected output has not been generated yet
// (iii) Its not timed out yet
System.clearProperty("spark.streaming.clock")
System.clearProperty("spark.driver.port")
ssc.start()
val startTime = System.currentTimeMillis()
while (!killed && !isLastOutputGenerated && !isTimedOut) {
Thread.sleep(100)
timeRan = System.currentTimeMillis() - startTime
isLastOutputGenerated = (!output.isEmpty && output.last == lastExpectedOutput)
isTimedOut = (timeRan + totalTimeRan > maxTimeToRun)
}
} catch {
case e: Exception => logError("Error running streaming context", e)
}
if (killingThread.isAlive) killingThread.interrupt()
ssc.stop()
logInfo("Has been killed = " + killed)
logInfo("Is last output generated = " + isLastOutputGenerated)
logInfo("Is timed out = " + isTimedOut)
// Verify whether the output of each batch has only one element or no element
// and then merge the new output with all the earlier output
mergedOutput ++= output
totalTimeRan += timeRan
logInfo("New output = " + output)
logInfo("Merged output = " + mergedOutput)
logInfo("Time ran = " + timeRan)
logInfo("Total time ran = " + totalTimeRan)
if (!isLastOutputGenerated && !isTimedOut) {
val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 10)
logInfo(
"\n-------------------------------------------\n" +
" Restarting stream computation in " + sleepTime + " ms " +
"\n-------------------------------------------\n"
)
Thread.sleep(sleepTime)
// Recreate the streaming context from checkpoint
ssc = new StreamingContext(checkpointDir)
}
}
mergedOutput
}
/**
* Verifies the output value are the same as expected. Since failures can lead to
* a batch being processed twice, a batches output may appear more than once
* consecutively. To avoid getting confused with those, we eliminate consecutive
* duplicate batch outputs of values from the `output`. As a result, the
* expected output should not have consecutive batches with the same values as output.
*/
private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) {
// Verify whether expected outputs do not consecutive batches with same output
for (i <- 0 until expectedOutput.size - 1) {
assert(expectedOutput(i) != expectedOutput(i+1),
"Expected output has consecutive duplicate sequence of values")
}
// Log the output
println("Expected output, size = " + expectedOutput.size)
println(expectedOutput.mkString("[", ",", "]"))
println("Output, size = " + output.size)
println(output.mkString("[", ",", "]"))
// Match the output with the expected output
output.foreach(o =>
assert(expectedOutput.contains(o), "Expected value " + o + " not found")
)
}
/** Resets counter to prepare for the test */
private def reset() {
killed = false
killCount = 0
}
}
/**
* This is a output stream just for testing. All the output is collected into a
* ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
*/
private[streaming]
class TestOutputStream[T: ClassManifest](
parent: DStream[T],
val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[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()
}
}
/**
* Thread to kill streaming context after a random period of time.
*/
private[streaming]
class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging {
initLogging()
override def run() {
try {
// If it is the first killing, then allow the first checkpoint to be created
var minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 2000
val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime)
logInfo("Kill wait time = " + killWaitTime)
Thread.sleep(killWaitTime)
logInfo(
"\n---------------------------------------\n" +
"Killing streaming context after " + killWaitTime + " ms" +
"\n---------------------------------------\n"
)
if (ssc != null) {
ssc.stop()
MasterFailureTest.killed = true
MasterFailureTest.killCount += 1
}
logInfo("Killing thread finished normally")
} catch {
case ie: InterruptedException => logInfo("Killing thread interrupted")
case e: Exception => logWarning("Exception in killing thread", e)
}
}
}
/**
* Thread to generate input files periodically with the desired text.
*/
private[streaming]
class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
extends Thread with Logging {
initLogging()
override def run() {
val localTestDir = Files.createTempDir()
var fs = testDir.getFileSystem(new Configuration())
val maxTries = 3
try {
Thread.sleep(5000) // To make sure that all the streaming context has been set up
for (i <- 0 until input.size) {
// Write the data to a local file and then move it to the target test directory
val localFile = new File(localTestDir, (i+1).toString)
val hadoopFile = new Path(testDir, (i+1).toString)
FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
var tries = 0
var done = false
while (!done && tries < maxTries) {
tries += 1
try {
fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
done = true
} catch {
case ioe: IOException => {
fs = testDir.getFileSystem(new Configuration())
logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
}
}
}
if (!done)
logError("Could not generate file " + hadoopFile)
else
logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis)
Thread.sleep(interval)
localFile.delete()
}
logInfo("File generating thread finished normally")
} catch {
case ie: InterruptedException => logInfo("File generating thread interrupted")
case e: Exception => logWarning("File generating in killing thread", e)
} finally {
fs.close()
}
}
}

View file

@ -3,9 +3,9 @@ package spark.streaming.util
private[streaming]
class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) {
val minPollTime = 25L
private val minPollTime = 25L
val pollTime = {
private val pollTime = {
if (period / 10.0 > minPollTime) {
(period / 10.0).toLong
} else {
@ -13,11 +13,20 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) =>
}
}
val thread = new Thread() {
private val thread = new Thread() {
override def run() { loop }
}
var nextTime = 0L
private var nextTime = 0L
def getStartTime(): Long = {
(math.floor(clock.currentTime.toDouble / period) + 1).toLong * period
}
def getRestartTime(originalStartTime: Long): Long = {
val gap = clock.currentTime - originalStartTime
(math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime
}
def start(startTime: Long): Long = {
nextTime = startTime
@ -26,21 +35,14 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) =>
}
def start(): Long = {
val startTime = (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period
start(startTime)
start(getStartTime())
}
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() {
def stop() {
thread.interrupt()
}
def loop() {
private def loop() {
try {
while (true) {
clock.waitTillTime(nextTime)

View file

@ -13,6 +13,8 @@ import scala.Tuple2;
import spark.HashPartitioner;
import spark.api.java.JavaPairRDD;
import spark.api.java.JavaRDD;
import spark.api.java.JavaRDDLike;
import spark.api.java.JavaPairRDD;
import spark.api.java.JavaSparkContext;
import spark.api.java.function.*;
import spark.storage.StorageLevel;
@ -22,10 +24,16 @@ import spark.streaming.api.java.JavaStreamingContext;
import spark.streaming.JavaTestUtils;
import spark.streaming.JavaCheckpointTestUtils;
import spark.streaming.dstream.KafkaPartitionKey;
import spark.streaming.InputStreamsSuite;
import java.io.*;
import java.util.*;
import akka.actor.Props;
import akka.zeromq.Subscribe;
// The test suite itself is Serializable so that anonymous Function implementations can be
// serialized, as an alternative to converting these anonymous classes to static inner classes;
// see http://stackoverflow.com/questions/758570/.
@ -34,8 +42,9 @@ public class JavaAPISuite implements Serializable {
@Before
public void setUp() {
ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
ssc.checkpoint("checkpoint", new Duration(1000));
System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock");
ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
ssc.checkpoint("checkpoint");
}
@After
@ -134,29 +143,6 @@ public class JavaAPISuite implements Serializable {
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(
@ -316,8 +302,9 @@ public class JavaAPISuite implements Serializable {
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>>() {
JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaDStream<Integer> 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>() {
@ -719,50 +706,73 @@ public class JavaAPISuite implements Serializable {
}
@Test
public void testCountByKey() {
List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
public void testCountByValue() {
List<List<String>> inputData = Arrays.asList(
Arrays.asList("hello", "world"),
Arrays.asList("hello", "moon"),
Arrays.asList("hello"));
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)));
Arrays.asList(
new Tuple2<String, Long>("hello", 1L),
new Tuple2<String, Long>("world", 1L)),
Arrays.asList(
new Tuple2<String, Long>("hello", 1L),
new Tuple2<String, Long>("moon", 1L)),
Arrays.asList(
new Tuple2<String, Long>("hello", 1L)));
JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
ssc, inputData, 1);
JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
JavaPairDStream<String, Long> counted = pairStream.countByKey();
JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaPairDStream<String, Long> counted = stream.countByValue();
JavaTestUtils.attachTestOutputStream(counted);
List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
Assert.assertEquals(expected, result);
}
@Test
public void testGroupByKeyAndWindow() {
List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
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"))));
List<List<Tuple2<String, List<Integer>>>> expected = Arrays.asList(
Arrays.asList(
new Tuple2<String, List<Integer>>("california", Arrays.asList(1, 3)),
new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 4))
),
Arrays.asList(
new Tuple2<String, List<Integer>>("california", Arrays.asList(1, 3, 5, 5)),
new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 1, 3, 4))
),
Arrays.asList(
new Tuple2<String, List<Integer>>("california", Arrays.asList(5, 5)),
new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 3))
)
);
JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
JavaPairDStream<String, List<String>> groupWindowed =
JavaPairDStream<String, List<Integer>> groupWindowed =
pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
JavaTestUtils.attachTestOutputStream(groupWindowed);
List<List<Tuple2<String, List<String>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
List<List<Tuple2<String, List<Integer>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
Assert.assertEquals(expected, result);
assert(result.size() == expected.size());
for (int i = 0; i < result.size(); i++) {
assert(convert(result.get(i)).equals(convert(expected.get(i))));
}
}
private HashSet<Tuple2<String, HashSet<Integer>>> convert(List<Tuple2<String, List<Integer>>> listOfTuples) {
List<Tuple2<String, HashSet<Integer>>> newListOfTuples = new ArrayList<Tuple2<String, HashSet<Integer>>>();
for (Tuple2<String, List<Integer>> tuple: listOfTuples) {
newListOfTuples.add(convert(tuple));
}
return new HashSet<Tuple2<String, HashSet<Integer>>>(newListOfTuples);
}
private Tuple2<String, HashSet<Integer>> convert(Tuple2<String, List<Integer>> tuple) {
return new Tuple2<String, HashSet<Integer>>(tuple._1(), new HashSet<Integer>(tuple._2()));
}
@Test
@ -847,26 +857,28 @@ public class JavaAPISuite implements Serializable {
}
@Test
public void testCountByKeyAndWindow() {
List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
public void testCountByValueAndWindow() {
List<List<String>> inputData = Arrays.asList(
Arrays.asList("hello", "world"),
Arrays.asList("hello", "moon"),
Arrays.asList("hello"));
List<List<Tuple2<String, Long>>> expected = Arrays.asList(
Arrays.asList(
new Tuple2<String, Long>("california", 2L),
new Tuple2<String, Long>("new york", 2L)),
new Tuple2<String, Long>("hello", 1L),
new Tuple2<String, Long>("world", 1L)),
Arrays.asList(
new Tuple2<String, Long>("california", 4L),
new Tuple2<String, Long>("new york", 4L)),
new Tuple2<String, Long>("hello", 2L),
new Tuple2<String, Long>("world", 1L),
new Tuple2<String, Long>("moon", 1L)),
Arrays.asList(
new Tuple2<String, Long>("california", 2L),
new Tuple2<String, Long>("new york", 2L)));
new Tuple2<String, Long>("hello", 2L),
new Tuple2<String, Long>("moon", 1L)));
JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
JavaDStream<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));
stream.countByValueAndWindow(new Duration(2000), new Duration(1000));
JavaTestUtils.attachTestOutputStream(counted);
List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
@ -918,6 +930,46 @@ public class JavaAPISuite implements Serializable {
}
@Test
public void testPairToNormalRDDTransform() {
List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
Arrays.asList(
new Tuple2<Integer, Integer>(3, 5),
new Tuple2<Integer, Integer>(1, 5),
new Tuple2<Integer, Integer>(4, 5),
new Tuple2<Integer, Integer>(2, 5)),
Arrays.asList(
new Tuple2<Integer, Integer>(2, 5),
new Tuple2<Integer, Integer>(3, 5),
new Tuple2<Integer, Integer>(4, 5),
new Tuple2<Integer, Integer>(1, 5)));
List<List<Integer>> expected = Arrays.asList(
Arrays.asList(3,1,4,2),
Arrays.asList(2,3,4,1));
JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
ssc, inputData, 1);
JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
JavaDStream<Integer> firstParts = pairStream.transform(
new Function<JavaPairRDD<Integer, Integer>, JavaRDD<Integer>>() {
@Override
public JavaRDD<Integer> call(JavaPairRDD<Integer, Integer> in) throws Exception {
return in.map(new Function<Tuple2<Integer, Integer>, Integer>() {
@Override
public Integer call(Tuple2<Integer, Integer> in) {
return in._1();
}
});
}
});
JavaTestUtils.attachTestOutputStream(firstParts);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
Assert.assertEquals(expected, result);
}
public void testMapValues() {
List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
@ -1091,9 +1143,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(1,4),
Arrays.asList(8,7));
File tempDir = Files.createTempDir();
ssc.checkpoint(tempDir.getAbsolutePath(), new Duration(1000));
ssc.checkpoint(tempDir.getAbsolutePath());
JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaDStream letterCount = stream.map(new Function<String, Integer>() {
@ -1107,14 +1158,16 @@ public class JavaAPISuite implements Serializable {
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);
// Tweak to take into consideration that the last batch before failure
// will be re-processed after recovery
List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3);
assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3));
}
/** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD
@Test
public void testCheckpointofIndividualStream() throws InterruptedException {
@ -1151,19 +1204,19 @@ public class JavaAPISuite implements Serializable {
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,
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);
public void testSocketTextStream() {
JavaDStream test = ssc.socketTextStream("localhost", 12345);
}
@Test
public void testNetworkString() {
public void testSocketString() {
class Converter extends Function<InputStream, Iterable<String>> {
public Iterable<String> call(InputStream in) {
BufferedReader reader = new BufferedReader(new InputStreamReader(in));
@ -1179,7 +1232,7 @@ public class JavaAPISuite implements Serializable {
}
}
JavaDStream test = ssc.networkStream(
JavaDStream test = ssc.socketStream(
"localhost",
12345,
new Converter(),
@ -1192,13 +1245,13 @@ public class JavaAPISuite implements Serializable {
}
@Test
public void testRawNetworkStream() {
JavaDStream test = ssc.rawNetworkStream("localhost", 12345);
public void testRawSocketStream() {
JavaDStream test = ssc.rawSocketStream("localhost", 12345);
}
@Test
public void testFlumeStream() {
JavaDStream test = ssc.flumeStream("localhost", 12345);
JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
}
@Test
@ -1206,4 +1259,25 @@ public class JavaAPISuite implements Serializable {
JavaPairDStream<String, String> foo =
ssc.<String, String, SequenceFileInputFormat>fileStream("/tmp/foo");
}
@Test
public void testTwitterStream() {
String[] filters = new String[] { "good", "bad", "ugly" };
JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY());
}
@Test
public void testActorStream() {
JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
}
@Test
public void testZeroMQStream() {
JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
@Override
public Iterable<String> call(byte[][] b) throws Exception {
return null;
}
});
}
}

View file

@ -31,8 +31,9 @@ trait JavaTestBase extends TestSuiteBase {
* Attach a provided stream to it's associated StreamingContext as a
* [[spark.streaming.TestOutputStream]].
**/
def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T,This]](
dstream: JavaDStreamLike[T, This]) = {
def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R],
R <: spark.api.java.JavaRDDLike[T, R]](
dstream: JavaDStreamLike[T, This, R]) = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
val ostream = new TestOutputStream(dstream.dstream,
@ -57,6 +58,7 @@ trait JavaTestBase extends TestSuiteBase {
}
object JavaTestUtils extends JavaTestBase {
override def maxWaitTimeMillis = 20000
}

View file

@ -1,5 +1,6 @@
# Set everything to be logged to the file streaming/target/unit-tests.log
log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
log4j.appender.file.file=streaming/target/unit-tests.log

View file

@ -6,6 +6,8 @@ import util.ManualClock
class BasicOperationsSuite extends TestSuiteBase {
System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
override def framework() = "BasicOperationsSuite"
after {
@ -22,7 +24,7 @@ class BasicOperationsSuite extends TestSuiteBase {
)
}
test("flatmap") {
test("flatMap") {
val input = Seq(1 to 4, 5 to 8, 9 to 12)
testOperation(
input,
@ -86,6 +88,23 @@ class BasicOperationsSuite extends TestSuiteBase {
)
}
test("count") {
testOperation(
Seq(1 to 1, 1 to 2, 1 to 3, 1 to 4),
(s: DStream[Int]) => s.count(),
Seq(Seq(1L), Seq(2L), Seq(3L), Seq(4L))
)
}
test("countByValue") {
testOperation(
Seq(1 to 1, Seq(1, 1, 1), 1 to 2, Seq(1, 1, 2, 2)),
(s: DStream[Int]) => s.countByValue(),
Seq(Seq((1, 1L)), Seq((1, 3L)), Seq((1, 1L), (2, 1L)), Seq((2, 2L), (1, 2L))),
true
)
}
test("mapValues") {
testOperation(
Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
@ -165,6 +184,71 @@ class BasicOperationsSuite extends TestSuiteBase {
testOperation(inputData, updateStateOperation, outputData, true)
}
test("updateStateByKey - object lifecycle") {
val inputData =
Seq(
Seq("a","b"),
null,
Seq("a","c","a"),
Seq("c"),
null,
null
)
val outputData =
Seq(
Seq(("a", 1), ("b", 1)),
Seq(("a", 1), ("b", 1)),
Seq(("a", 3), ("c", 1)),
Seq(("a", 3), ("c", 2)),
Seq(("c", 2)),
Seq()
)
val updateStateOperation = (s: DStream[String]) => {
class StateObject(var counter: Int = 0, var expireCounter: Int = 0) extends Serializable
// updateFunc clears a state when a StateObject is seen without new values twice in a row
val updateFunc = (values: Seq[Int], state: Option[StateObject]) => {
val stateObj = state.getOrElse(new StateObject)
values.foldLeft(0)(_ + _) match {
case 0 => stateObj.expireCounter += 1 // no new values
case n => { // has new values, increment and reset expireCounter
stateObj.counter += n
stateObj.expireCounter = 0
}
}
stateObj.expireCounter match {
case 2 => None // seen twice with no new values, give it the boot
case _ => Option(stateObj)
}
}
s.map(x => (x, 1)).updateStateByKey[StateObject](updateFunc).mapValues(_.counter)
}
testOperation(inputData, updateStateOperation, outputData, true)
}
test("slice") {
val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
val stream = new TestInputStream[Int](ssc, input, 2)
ssc.registerInputStream(stream)
stream.foreach(_ => {}) // Dummy output stream
ssc.start()
Thread.sleep(2000)
def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
}
assert(getInputFromSlice(0, 1000) == Set(1))
assert(getInputFromSlice(0, 2000) == Set(1, 2))
assert(getInputFromSlice(1000, 2000) == Set(1, 2))
assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
ssc.stop()
Thread.sleep(1000)
}
test("forgetting of RDDs - map and window operations") {
assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second")

View file

@ -1,5 +1,6 @@
package spark.streaming
import dstream.FileInputDStream
import spark.streaming.StreamingContext._
import java.io.File
import runtime.RichInt
@ -7,9 +8,19 @@ import org.scalatest.BeforeAndAfter
import org.apache.commons.io.FileUtils
import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
import util.{Clock, ManualClock}
import scala.util.Random
import com.google.common.io.Files
/**
* This test suites tests the checkpointing functionality of DStreams -
* the checkpointing of a DStream's RDDs as well as the checkpointing of
* the whole DStream graph.
*/
class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
before {
FileUtils.deleteDirectory(new File(checkpointDir))
}
@ -28,21 +39,18 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
override def batchDuration = Milliseconds(500)
override def checkpointInterval = batchDuration
override def actuallyWait = true
test("basic stream+rdd recovery") {
test("basic rdd checkpoints + dstream graph checkpoint 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 firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
val secondNumBatches = firstNumBatches
// Setup the streams
@ -62,10 +70,10 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// 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 {
advanceTimeWithRealDelay(ssc, firstNumBatches)
logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData)
assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure")
stateStream.checkpointData.checkpointFiles.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")
@ -74,8 +82,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// 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)
val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2))
advanceTimeWithRealDelay(ssc, secondNumBatches)
checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
ssc.stop()
@ -90,9 +98,9 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// 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 {
advanceTimeWithRealDelay(ssc, 1)
assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure")
stateStream.checkpointData.checkpointFiles.foreach {
case (time, data) => {
val file = new File(data.toString)
assert(file.exists(),
@ -111,13 +119,16 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// 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)
advanceTimeWithRealDelay(ssc, 4)
ssc.stop()
System.clearProperty("spark.streaming.manualClock.jump")
ssc = null
}
test("map and reduceByKey") {
// This tests whether the systm can recover from a master failure with simple
// non-stateful operations. This assumes as reliable, replayable input
// source - TestInputDStream.
test("recovery with map and reduceByKey operations") {
testCheckpointedOperation(
Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ),
(s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _),
@ -126,7 +137,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
)
}
test("reduceByKeyAndWindowInv") {
// This tests whether the ReduceWindowedDStream's RDD checkpoints works correctly such
// that the system can recover from a master failure. This assumes as reliable,
// replayable input source - TestInputDStream.
test("recovery with invertible reduceByKeyAndWindow operation") {
val n = 10
val w = 4
val input = (1 to n).map(_ => Seq("a")).toSeq
@ -139,7 +154,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
testCheckpointedOperation(input, operation, output, 7)
}
test("updateStateByKey") {
// This tests whether the StateDStream's RDD checkpoints works correctly such
// that the system can recover from a master failure. This assumes as reliable,
// replayable input source - TestInputDStream.
test("recovery with updateStateByKey operation") {
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]) => {
@ -154,11 +173,126 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
testCheckpointedOperation(input, operation, output, 7)
}
// This tests whether file input stream remembers what files were seen before
// the master failure and uses them again to process a large window operation.
// It also tests whether batches, whose processing was incomplete due to the
// failure, are re-processed or not.
test("recovery with file input stream") {
// Disable manual clock as FileInputDStream does not work with manual clock
val clockProperty = System.getProperty("spark.streaming.clock")
System.clearProperty("spark.streaming.clock")
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
var ssc = new StreamingContext(master, framework, Seconds(1))
ssc.checkpoint(checkpointDir)
val fileStream = ssc.textFileStream(testDir.toString)
// Making value 3 take large time to process, to ensure that the master
// shuts down in the middle of processing the 3rd batch
val mappedStream = fileStream.map(s => {
val i = s.toInt
if (i == 3) Thread.sleep(2000)
i
})
// Reducing over a large window to ensure that recovery from master failure
// requires reprocessing of all the files seen before the failure
val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1))
val outputBuffer = new ArrayBuffer[Seq[Int]]
var outputStream = new TestOutputStream(reducedStream, outputBuffer)
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")
// wait to make sure that the file is written such that it gets shown in the file listings
Thread.sleep(1000)
}
logInfo("Output = " + outputStream.output.mkString(","))
assert(outputStream.output.size > 0, "No files processed before restart")
ssc.stop()
// Verify whether files created have been recorded correctly or not
var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]]
def recordedFiles = fileInputDStream.files.values.flatMap(x => x)
assert(!recordedFiles.filter(_.endsWith("1")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("2")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("3")).isEmpty)
// Create files while the master is down
for (i <- Seq(4, 5, 6)) {
FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
Thread.sleep(1000)
}
// Recover context from checkpoint file and verify whether the files that were
// recorded before failure were saved and successfully recovered
logInfo("*********** RESTARTING ************")
ssc = new StreamingContext(checkpointDir)
fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]]
assert(!recordedFiles.filter(_.endsWith("1")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("2")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("3")).isEmpty)
// Restart stream computation
ssc.start()
for (i <- Seq(7, 8, 9)) {
FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
Thread.sleep(1000)
}
Thread.sleep(1000)
logInfo("Output = " + outputStream.output.mkString("[", ", ", "]"))
assert(outputStream.output.size > 0, "No files processed after restart")
ssc.stop()
// Verify whether files created while the driver was down have been recorded or not
assert(!recordedFiles.filter(_.endsWith("4")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("5")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("6")).isEmpty)
// Verify whether new files created after recover have been recorded or not
assert(!recordedFiles.filter(_.endsWith("7")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("8")).isEmpty)
assert(!recordedFiles.filter(_.endsWith("9")).isEmpty)
// Append the new output to the old buffer
outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]]
outputBuffer ++= outputStream.output
val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45)
logInfo("--------------------------------")
logInfo("output, size = " + outputBuffer.size)
outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
logInfo("expected output, size = " + expectedOutput.size)
expectedOutput.foreach(x => logInfo("[" + x + "]"))
logInfo("--------------------------------")
// Verify whether all the elements received are as expected
val output = outputBuffer.flatMap(x => x)
assert(output.contains(6)) // To ensure that the 3rd input (i.e., 3) was processed
output.foreach(o => // To ensure all the inputs are correctly added cumulatively
assert(expectedOutput.contains(o), "Expected value " + o + " not found")
)
// To ensure that all the inputs were received correctly
assert(expectedOutput.last === output.last)
// Enable manual clock back again for other tests
if (clockProperty != null)
System.setProperty("spark.streaming.clock", clockProperty)
}
/**
* Tests a streaming operation under checkpointing, by restart the operation
* Tests a streaming operation under checkpointing, by restarting 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.
*
* NOTE: This takes into consideration that the last batch processed before
* master failure will be re-processed after restart/recovery.
*/
def testCheckpointedOperation[U: ClassManifest, V: ClassManifest](
input: Seq[Seq[U]],
@ -172,11 +306,14 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
val totalNumBatches = input.size
val nextNumBatches = totalNumBatches - initialNumBatches
val initialNumExpectedOutputs = initialNumBatches
val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs
val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs + 1
// because the last batch will be processed again
// 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)
ssc.start()
val output = advanceTimeWithRealDelay[V](ssc, initialNumBatches)
ssc.stop()
verifyOutput[V](output, expectedOutput.take(initialNumBatches), true)
Thread.sleep(1000)
@ -187,16 +324,20 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
"\n-------------------------------------------\n"
)
ssc = new StreamingContext(checkpointDir)
val outputNew = runStreams[V](ssc, nextNumBatches, nextNumExpectedOutputs)
System.clearProperty("spark.driver.port")
ssc.start()
val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches)
// the first element will be re-processed data of the last batch before restart
verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true)
ssc.stop()
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.
* It also waits for the expected amount of time for each batch.
*/
def runStreamsWithRealDelay(ssc: StreamingContext, numBatches: Long) {
def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
logInfo("Manual clock before advancing = " + clock.time)
for (i <- 1 to numBatches.toInt) {
@ -205,6 +346,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
}
logInfo("Manual clock after advancing = " + clock.time)
Thread.sleep(batchDuration.milliseconds)
}
val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
outputStream.output
}
}

View file

@ -1,191 +1,40 @@
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
import spark.streaming.util.MasterFailureTest
import StreamingContext._
import org.scalatest.{FunSuite, BeforeAndAfter}
import com.google.common.io.Files
import java.io.File
import org.apache.commons.io.FileUtils
import collection.mutable.ArrayBuffer
/**
* This testsuite tests master failures at random times while the stream is running using
* the real clock.
*/
class FailureSuite extends TestSuiteBase with BeforeAndAfter {
class FailureSuite extends FunSuite with BeforeAndAfter with Logging {
var directory = "FailureSuite"
val numBatches = 30
val batchDuration = Milliseconds(1000)
before {
FileUtils.deleteDirectory(new File(checkpointDir))
FileUtils.deleteDirectory(new File(directory))
}
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.driver.port")
FileUtils.deleteDirectory(new File(directory))
}
override def framework = "CheckpointSuite"
override def batchDuration = Milliseconds(500)
override def checkpointDir = "checkpoint"
override def checkpointInterval = batchDuration
test("multiple failures with map") {
MasterFailureTest.testMap(directory, numBatches, 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
MasterFailureTest.testUpdateStateByKey(directory, numBatches, batchDuration)
}
}
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
}
}

View file

@ -1,5 +1,11 @@
package spark.streaming
import akka.actor.Actor
import akka.actor.IO
import akka.actor.IOManager
import akka.actor.Props
import akka.util.ByteString
import dstream.SparkFlumeEvent
import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
import java.io.{File, BufferedWriter, OutputStreamWriter}
@ -7,6 +13,7 @@ import java.util.concurrent.{TimeUnit, ArrayBlockingQueue}
import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
import util.ManualClock
import spark.storage.StorageLevel
import spark.streaming.receivers.Receiver
import spark.Logging
import scala.util.Random
import org.apache.commons.io.FileUtils
@ -19,40 +26,30 @@ import org.apache.avro.ipc.specific.SpecificRequestor
import java.nio.ByteBuffer
import collection.JavaConversions._
import java.nio.charset.Charset
import com.google.common.io.Files
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.driver.port")
}
test("network input stream") {
test("socket input stream") {
// Start the server
testServer = new TestServer(testPort)
val 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 networkStream = ssc.socketTextStream("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)
@ -93,46 +90,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
}
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
@ -146,7 +103,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
val input = Seq(1, 2, 3, 4, 5)
Thread.sleep(1000)
val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", 33333));
val client = SpecificRequestor.getClient(
classOf[AvroSourceProtocol], transceiver);
@ -182,42 +139,33 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
}
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
}
test("file input stream") {
// Disable manual clock as FileInputDStream does not work with manual clock
System.clearProperty("spark.streaming.clock")
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
val ssc = new StreamingContext(master, framework, batchDuration)
val filestream = ssc.textFileStream(testDir.toString)
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)
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 file = new File(testDir, i.toString)
FileUtils.writeStringToFile(file, input(i).toString + "\n")
logInfo("Created file " + file)
Thread.sleep(batchDuration.milliseconds)
Thread.sleep(1000)
}
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")
@ -226,6 +174,57 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
// Verify whether data received by Spark Streaming was as expected
logInfo("--------------------------------")
logInfo("output, size = " + outputBuffer.size)
outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
logInfo("expected output, size = " + expectedOutput.size)
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.toList === expectedOutput.toList)
FileUtils.deleteDirectory(testDir)
// Enable manual clock back again for other tests
System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
}
test("actor input stream") {
// Start the server
val port = testPort
val testServer = new TestServer(port)
testServer.start()
// Set up the streaming context and input streams
val ssc = new StreamingContext(master, framework, batchDuration)
val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor",
StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
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 = 1 to 9
val expectedOutput = input.map(x => x.toString)
Thread.sleep(1000)
for (i <- 0 until input.size) {
testServer.send(input(i).toString)
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(",") + "]"))
@ -238,63 +237,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
// (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))
assert(output(i) === 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()
}
}
/** This is server to test the network input stream */
class TestServer(port: Int) extends Logging {
val queue = new ArrayBlockingQueue[String](100)
@ -353,3 +302,15 @@ object TestServer {
}
}
}
class TestActor(port: Int) extends Actor with Receiver {
def bytesToString(byteString: ByteString) = byteString.utf8String
override def preStart = IOManager(context.system).connect(new InetSocketAddress(port))
def receive = {
case IO.Read(socket, bytes) =>
pushBlock(bytesToString(bytes))
}
}

View file

@ -28,6 +28,11 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
logInfo("Computing RDD for time " + validTime)
val index = ((validTime - zeroTime) / slideDuration - 1).toInt
val selectedInput = if (index < input.size) input(index) else Seq[T]()
// lets us test cases where RDDs are not created
if (selectedInput == null)
return None
val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
logInfo("Created RDD " + rdd.id + " with " + selectedInput)
Some(rdd)
@ -58,20 +63,25 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBu
*/
trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Name of the framework for Spark context
def framework = "TestSuiteBase"
// Master for Spark context
def master = "local[2]"
// Batch duration
def batchDuration = Seconds(1)
// Directory where the checkpoint data will be saved
def checkpointDir = "checkpoint"
def checkpointInterval = batchDuration
// Number of partitions of the input parallel collections created for testing
def numInputPartitions = 2
// Maximum time to wait before the test times out
def maxWaitTimeMillis = 10000
// Whether to actually wait in real time before changing manual clock
def actuallyWait = false
/**
@ -86,7 +96,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Create StreamingContext
val ssc = new StreamingContext(master, framework, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir, checkpointInterval)
ssc.checkpoint(checkpointDir)
}
// Setup the stream computation
@ -111,7 +121,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Create StreamingContext
val ssc = new StreamingContext(master, framework, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir, checkpointInterval)
ssc.checkpoint(checkpointDir)
}
// Setup the stream computation
@ -135,9 +145,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
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)
@ -181,7 +188,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
} finally {
ssc.stop()
}
output
}

View file

@ -5,6 +5,8 @@ import collection.mutable.ArrayBuffer
class WindowOperationsSuite extends TestSuiteBase {
System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
override def framework = "WindowOperationsSuite"
override def maxWaitTimeMillis = 20000
@ -82,12 +84,9 @@ class WindowOperationsSuite extends TestSuiteBase {
)
/*
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.
The output of the reduceByKeyAndWindow with inverse function but without a filter
function will be different from the naive reduceByKeyAndWindow, as no keys get
eliminated from the ReducedWindowedDStream even if the value of a key becomes 0.
*/
val bigReduceInvOutput = Seq(
@ -175,31 +174,31 @@ class WindowOperationsSuite extends TestSuiteBase {
// Testing reduceByKeyAndWindow (with invertible reduce function)
testReduceByKeyAndWindowInv(
testReduceByKeyAndWindowWithInverse(
"basic reduction",
Seq(Seq(("a", 1), ("a", 3)) ),
Seq(Seq(("a", 4)) )
)
testReduceByKeyAndWindowInv(
testReduceByKeyAndWindowWithInverse(
"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(
testReduceByKeyAndWindowWithInverse(
"new key added into window",
Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
)
testReduceByKeyAndWindowInv(
testReduceByKeyAndWindowWithInverse(
"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(
testReduceByKeyAndWindowWithInverse(
"larger slide time",
largerSlideInput,
largerSlideReduceOutput,
@ -207,7 +206,9 @@ class WindowOperationsSuite extends TestSuiteBase {
Seconds(2)
)
testReduceByKeyAndWindowInv("big test", bigInput, bigReduceInvOutput)
testReduceByKeyAndWindowWithInverse("big test", bigInput, bigReduceInvOutput)
testReduceByKeyAndWindowWithFilteredInverse("big test", bigInput, bigReduceOutput)
test("groupByKeyAndWindow") {
val input = bigInput
@ -235,14 +236,14 @@ class WindowOperationsSuite extends TestSuiteBase {
testOperation(input, operation, expectedOutput, numBatches, true)
}
test("countByKeyAndWindow") {
val input = Seq(Seq(("a", 1)), Seq(("b", 1), ("b", 2)), Seq(("a", 10), ("b", 20)))
test("countByValueAndWindow") {
val input = Seq(Seq("a"), Seq("b", "b"), Seq("a", "b"))
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))
val operation = (s: DStream[String]) => {
s.countByValueAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt))
}
testOperation(input, operation, expectedOutput, numBatches, true)
}
@ -272,29 +273,50 @@ class WindowOperationsSuite extends TestSuiteBase {
slideDuration: Duration = Seconds(1)
) {
test("reduceByKeyAndWindow - " + name) {
logInfo("reduceByKeyAndWindow - " + name)
val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
val operation = (s: DStream[(String, Int)]) => {
s.reduceByKeyAndWindow(_ + _, windowDuration, slideDuration).persist()
s.reduceByKeyAndWindow((x: Int, y: Int) => x + y, windowDuration, slideDuration)
}
testOperation(input, operation, expectedOutput, numBatches, true)
}
}
def testReduceByKeyAndWindowInv(
def testReduceByKeyAndWindowWithInverse(
name: String,
input: Seq[Seq[(String, Int)]],
expectedOutput: Seq[Seq[(String, Int)]],
windowDuration: Duration = Seconds(2),
slideDuration: Duration = Seconds(1)
) {
test("reduceByKeyAndWindowInv - " + name) {
test("reduceByKeyAndWindow with inverse function - " + name) {
logInfo("reduceByKeyAndWindow with inverse function - " + 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)
}
}
def testReduceByKeyAndWindowWithFilteredInverse(
name: String,
input: Seq[Seq[(String, Int)]],
expectedOutput: Seq[Seq[(String, Int)]],
windowDuration: Duration = Seconds(2),
slideDuration: Duration = Seconds(1)
) {
test("reduceByKeyAndWindow with inverse and filter functions - " + name) {
logInfo("reduceByKeyAndWindow with inverse and filter functions - " + name)
val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
val filterFunc = (p: (String, Int)) => p._2 != 0
val operation = (s: DStream[(String, Int)]) => {
s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration, filterFunc = filterFunc)
.persist()
.checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing
}
testOperation(input, operation, expectedOutput, numBatches, true)
}
}
}