--- layout: global title: Spark Streaming Programming Guide --- * This will become a table of contents (this text will be scraped). {:toc} # Overview A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collection of elements) representing a continuous stream of data. DStreams can created from live incoming data (such as data from a socket, Kafka, etc.) or it can be generated by transformation of existing DStreams using parallel operators like map, reduce, and window. The basic processing model is as follows: (i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides the data into batches. Each batch of data is treated as a RDD, that is a immutable and parallel collection of data. These input data RDDs are automatically persisted in memory (serialized by default) and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively referred to as an InputDStream. (ii) Data received by InputDStreams are processed processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. This guide shows some how to start programming with DStreams. # Initializing Spark Streaming The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using {% highlight scala %} new StreamingContext(master, jobName, batchDuration) {% endhighlight %} The `master` parameter is either the [Mesos master URL](running-on-mesos.html) (for running on a cluster)or the special "local" string (for local mode) that is used to create a Spark Context. For more information about this please refer to the [Spark programming guide](scala-programming-guide.html). The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the Mesos web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds maybe a good start. See [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using {% highlight scala %} new StreamingContext(sparkContext, batchDuration) {% endhighlight %} # Attaching Input Sources - InputDStreams The StreamingContext is used to creating InputDStreams from input sources: {% highlight scala %} // Assuming ssc is the StreamingContext ssc.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 %} 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 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:
Transformation | Meaning |
---|---|
map(func) | Returns a new DStream formed by passing each element of the source DStream through a function func. |
filter(func) | Returns a new DStream formed by selecting those elements of the source DStream on which func returns true. |
flatMap(func) | Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item). |
mapPartitions(func) | Similar to map, but runs separately on each partition (block) of the DStream, so func must be of type Iterator[T] => Iterator[U] when running on an DStream of type T. |
union(otherStream) | Return a new DStream that contains the union of the elements in the source DStream and the argument DStream. |
count() | Returns a new DStream of single-element RDDs by counting the number of elements in each RDD of the source DStream. |
reduce(func) | Returns a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed in parallel. |
countByValue() | 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. |
groupByKey([numTasks]) | 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. Note: 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 numTasks argument to set a different number of tasks.
|
reduceByKey(func, [numTasks]) | 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 groupByKey , the number of reduce tasks is configurable through an optional second argument. |
join(otherStream, [numTasks]) | 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. |
cogroup(otherStream, [numTasks]) | When called on DStream of (K, V) and (K, W) pairs, returns a new DStream of (K, Seq[V], Seq[W]) tuples. |
transform(func) | 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. |
updateStateByKey(func) | 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. |
Transformation | Meaning |
---|---|
window(windowDuration, slideDuration) | Return a new DStream which is computed based on windowed batches of the source DStream. windowDuration is the width of the window and slideTime is the frequency during which the window is calculated. Both times must be multiples of the batch interval. |
countByWindow(windowDuration, slideDuration) | Return a sliding count of elements in the stream. windowDuration and slideDuration are exactly as defined in window() .
|
reduceByWindow(func, windowDuration, slideDuration) | Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using func. The function should be associative so that it can be computed correctly in parallel. windowDuration and slideDuration are exactly as defined in window() .
|
groupByKeyAndWindow(windowDuration, slideDuration, [numTasks]) | 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. Note: 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 numTasks argument to set a different number of tasks. |
reduceByKeyAndWindow(func, windowDuration, slideDuration, [numTasks]) | 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 func over batches in a sliding window. Like in groupByKeyAndWindow , the number of reduce tasks is configurable through an optional second argument.
windowDuration and slideDuration are exactly as defined in window() .
|
reduceByKeyAndWindow(func, invFunc, windowDuration, slideDuration, [numTasks]) | A more efficient version of the above reduceByKeyAndWindow() 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 invFunc. Like in groupByKeyAndWindow , the number of reduce tasks is configurable through an optional second argument.
windowDuration and slideDuration are exactly as defined in window() .
|
countByValueAndWindow(windowDuration, slideDuration, [numTasks]) | 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 groupByKeyAndWindow , the number of reduce tasks is configurable through an optional second argument.
windowDuration and slideDuration are exactly as defined in window() .
|
Operator | Meaning |
---|---|
foreach(func) | The fundamental output operator. Applies a function, func, 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. |
print() | Prints first ten elements of every batch of data in a DStream on the driver. |
saveAsObjectFiles(prefix, [suffix]) | Save this DStream's contents as a SequenceFile of serialized objects. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
|
saveAsTextFiles(prefix, [suffix]) | Save this DStream's contents as a text files. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]". |
saveAsHadoopFiles(prefix, [suffix]) | Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]". |
{% highlight bash %} # TERMINAL 1 # RUNNING NETCAT $ nc -lk 9999 hello world ... {% endhighlight %} | {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount ... 2012-12-31 18:47:10,446 INFO SparkContext: Job finished: run at ThreadPoolExecutor.java:886, took 0.038817 s ------------------------------------------- Time: 1357008430000 ms ------------------------------------------- (hello,1) (world,1) 2012-12-31 18:47:10,447 INFO JobManager: Total delay: 0.44700 s for job 8 (execution: 0.44000 s) ... {% endhighlight %} |
Time | Number of lines in input file | Output without driver failure | Output with driver failure |
---|---|---|---|
1 | 10 | 10 | 10 |
2 | 20 | 20 | 20 |
3 | 30 | 30 | 30 |
4 | 40 | 40 | [DRIVER FAILS] no output |
5 | 50 | 50 | no output |
6 | 60 | 60 | no output |
7 | 70 | 70 | [DRIVER RECOVERS] 40, 50, 60, 70 |
8 | 80 | 80 | 80 |
9 | 90 | 90 | 90 |
10 | 100 | 100 | 100 |