aboutsummaryrefslogtreecommitdiff
path: root/docs/streaming-programming-guide.md
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2014-01-28 21:51:05 -0800
committerPatrick Wendell <pwendell@gmail.com>2014-01-28 21:51:05 -0800
commit793020961489e16e924c4531da3a13884d2b9175 (patch)
tree54da8edfdcf82072fbd9fb238e653da5b7849448 /docs/streaming-programming-guide.md
parentf8c742ce274fbae2a9e616d4c97469b6a22069bb (diff)
downloadspark-793020961489e16e924c4531da3a13884d2b9175.tar.gz
spark-793020961489e16e924c4531da3a13884d2b9175.tar.bz2
spark-793020961489e16e924c4531da3a13884d2b9175.zip
Merge pull request #497 from tdas/docs-update
Updated Spark Streaming Programming Guide Here is the updated version of the Spark Streaming Programming Guide. This is still a work in progress, but the major changes are in place. So feedback is most welcome. In general, I have tried to make the guide to easier to understand even if the reader does not know much about Spark. The updated website is hosted here - http://www.eecs.berkeley.edu/~tdas/spark_docs/streaming-programming-guide.html The major changes are: - Overview illustrates the usecases of Spark Streaming - various input sources and various output sources - An example right after overview to quickly give an idea of what Spark Streaming program looks like - Made Java API and examples a first class citizen like Scala by using tabs to show both Scala and Java examples (similar to AMPCamp tutorial's code tabs) - Highlighted the DStream operations updateStateByKey and transform because of their powerful nature - Updated driver node failure recovery text to highlight automatic recovery in Spark standalone mode - Added information about linking and using the external input sources like Kafka and Flume - In general, reorganized the sections to better show the Basic section and the more advanced sections like Tuning and Recovery. Todos: - Links to the docs of external Kafka, Flume, etc - Illustrate window operation with figure as well as example. Author: Tathagata Das <tathagata.das1565@gmail.com> == Merge branch commits == commit 18ff10556570b39d672beeb0a32075215cfcc944 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 21:49:30 2014 -0800 Fixed a lot of broken links. commit 34a5a6008dac2e107624c7ff0db0824ee5bae45f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 18:02:28 2014 -0800 Updated github url to use SPARK_GITHUB_URL variable. commit f338a60ae8069e0a382d2cb170227e5757cc0b7a Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 22:42:42 2014 -0800 More updates based on Patrick and Harvey's comments. commit 89a81ff25726bf6d26163e0dd938290a79582c0f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 13:08:34 2014 -0800 Updated docs based on Patricks PR comments. commit d5b6196b532b5746e019b959a79ea0cc013a8fc3 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 20:15:58 2014 -0800 Added spark.streaming.unpersist config and info on StreamingListener interface. commit e3dcb46ab83d7071f611d9b5008ba6bc16c9f951 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 18:41:12 2014 -0800 Fixed docs on StreamingContext.getOrCreate. commit 6c29524639463f11eec721e4d17a9d7159f2944b Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Thu Jan 23 18:49:39 2014 -0800 Added example and figure for window operations, and links to Kafka and Flume API docs. commit f06b964a51bb3b21cde2ff8bdea7d9785f6ce3a9 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:49:12 2014 -0800 Fixed missing endhighlight tag in the MLlib guide. commit 036a7d46187ea3f2a0fb8349ef78f10d6c0b43a9 Merge: eab351d a1cd185 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:42 2014 -0800 Merge remote-tracking branch 'apache/master' into docs-update commit eab351d05c0baef1d4b549e1581310087158d78d Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:15 2014 -0800 Update Spark Streaming Programming Guide.
Diffstat (limited to 'docs/streaming-programming-guide.md')
-rw-r--r--docs/streaming-programming-guide.md1278
1 files changed, 988 insertions, 290 deletions
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 07c4c55633..ccc1c8bf2c 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -7,74 +7,457 @@ title: Spark Streaming Programming Guide
{: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 collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transforming 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 it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream.
-(ii) Data received by InputDStreams are processed 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.
+Spark Streaming is an extension of the core Spark API that allows enables high-throughput,
+fault-tolerant stream processing of live data streams. Data can be ingested from many sources
+like Kafka, Flume, Twitter, ZeroMQ or plain old TCP sockets and be processed using complex
+algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`.
+Finally, processed data can be pushed out to filesystems, databases,
+and live dashboards. In fact, you can apply Spark's in-built
+[machine learning](mllib-guide.html) algorithms, and
+[graph processing](graphx-programming-guide.html) algorithms on data streams.
+
+<p style="text-align: center;">
+ <img
+ src="img/streaming-arch.png"
+ title="Spark Streaming architecture"
+ alt="Spark Streaming"
+ width="70%"
+ />
+</p>
+
+Internally, it works as follows. Spark Streaming receives live input data streams and divides
+the data into batches, which are then processed by the Spark engine to generate the final
+stream of results in batches.
+
+<p style="text-align: center;">
+ <img src="img/streaming-flow.png"
+ title="Spark Streaming data flow"
+ alt="Spark Streaming"
+ width="70%" />
+</p>
+
+Spark Streaming provides a high-level abstraction called *discretized stream* or *DStream*,
+which represents a continuous stream of data. DStreams can be created either from input data
+stream from sources such as Kafka and Flume, or by applying high-level
+operations on other DStreams. Internally, a DStream is represented as a sequence of
+[RDDs](api/core/index.html#org.apache.spark.rdd.RDD).
+
+This guide shows you how to start writing Spark Streaming programs with DStreams. You can
+write Spark Streaming programs in Scala or Java, both of which are presented in this guide. You
+will find tabs throughout this guide that let you choose between Scala and Java
+code snippets.
+
+***************************************************************************************************
+
+# A Quick Example
+Before we go into the details of how to write your own Spark Streaming program,
+let's take a quick look at what a simple Spark Streaming program looks like. Let's say we want to
+count the number of words in text data received from a data server listening on a TCP
+socket. All you need to
+do is as follows.
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1" >
+
+First, we create a
+[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object,
+which is the main entry point for all streaming
+functionality. Besides Spark's configuration, we specify that any DStream will be processed
+in 1 second batches.
-This guide shows some how to start programming with DStreams.
+{% highlight scala %}
+// Create a StreamingContext with a SparkConf configuration
+val ssc = new StreamingContext(sparkConf, Seconds(1))
+{% endhighlight %}
-# Linking with Spark Streaming
+Using this context, we then create a new DStream
+by specifying the IP address and port of the data server.
-Add the following SBT or Maven dependency to your project to use Spark Streaming:
+{% highlight scala %}
+// Create a DStream that will connect to serverIP:serverPort
+val lines = ssc.socketTextStream(serverIP, serverPort)
+{% endhighlight %}
- groupId = org.apache.spark
- artifactId = spark-streaming_{{site.SCALA_VERSION}}
- version = {{site.SPARK_VERSION}}
+This `lines` DStream represents the stream of data that will be received from the data
+server. Each record in this DStream is a line of text. Next, we want to split the lines by
+space into words.
-For ingesting data from sources like Kafka and Flume, add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_VERSION}}` to the dependencies. For example, `spark-streaming-kafka_{{site.SCALA_VERSION}}` for Kafka, `spark-streaming-flume_{{site.SCALA_VERSION}}`, etc. Please refer to the [Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION}}%22) for the full list of supported sources / artifacts.
+{% highlight scala %}
+// Split each line into words
+val words = lines.flatMap(_.split(" "))
+{% endhighlight %}
-# 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
+`flatMap` is a one-to-many DStream operation that creates a new DStream by
+generating multiple new records from each record int the source DStream. In this case,
+each line will be split into multiple words and the stream of words is represented as the
+`words` DStream. Next, we want to count these words.
{% highlight scala %}
-new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
+// Count each word in each batch
+val pairs = words.map(word => (word, 1))
+val wordCounts = pairs.reduceByKey(_ + _)
+
+// Print a few of the counts to the console
+wordCount.print()
{% endhighlight %}
-The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are optional parameters, which need to be set when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
+The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word,
+1)` pairs, which is then reduced to get the frequency of words in each batch of data.
+Finally, `wordCounts.print()` will print a few of the counts generated every second.
+
+Note that when these lines are executed, Spark Streaming only sets up the computation it
+will perform when it is started, and no real processing has started yet. To start the processing
+after all the transformations have been setup, we finally call
{% highlight scala %}
-new SparkConf(conf, batchDuration)
+ssc.start() // Start the computation
+ssc.awaitTermination() // Wait for the computation to terminate
+{% endhighlight %}
+
+The complete code can be found in the Spark Streaming example
+[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala).
+<br>
+
+</div>
+<div data-lang="java" markdown="1">
+
+First, we create a
+[JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object,
+which is the main entry point for all streaming
+functionality. Besides Spark's configuration, we specify that any DStream would be processed
+in 1 second batches.
+
+{% highlight java %}
+// Create a StreamingContext with a SparkConf configuration
+JavaStreamingContext jssc = StreamingContext(sparkConf, new Duration(1000))
+{% endhighlight %}
+
+Using this context, we then create a new DStream
+by specifying the IP address and port of the data server.
+
+{% highlight java %}
+// Create a DStream that will connect to serverIP:serverPort
+JavaDStream<String> lines = jssc.socketTextStream(serverIP, serverPort);
+{% endhighlight %}
+
+This `lines` DStream represents the stream of data that will be received from the data
+server. Each record in this stream is a line of text. Then, we want to split the the lines by
+space into words.
+
+{% highlight java %}
+// Split each line into words
+JavaDStream<String> words = lines.flatMap(
+ new FlatMapFunction<String, String>() {
+ @Override public Iterable<String> call(String x) {
+ return Lists.newArrayList(x.split(" "));
+ }
+ });
+{% endhighlight %}
+
+`flatMap` is a DStream operation that creates a new DStream by
+generating multiple new records from each record in the source DStream. In this case,
+each line will be split into multiple words and the stream of words is represented as the
+`words` DStream. Note that we defined the transformation using a
+[FlatMapFunction](api/core/index.html#org.apache.spark.api.java.function.FlatMapFunction) object.
+As we will discover along the way, there are a number of such convenience classes in the Java API
+that help define DStream transformations.
+
+Next, we want to count these words.
+
+{% highlight java %}
+// Count each word in each batch
+JavaPairDStream<String, Integer> pairs = words.map(
+ new PairFunction<String, String, Integer>() {
+ @Override public Tuple2<String, Integer> call(String s) throws Exception {
+ return new Tuple2<String, Integer>(s, 1);
+ }
+ });
+JavaPairDStream<String, Integer> wordCounts = pairs.reduceByKey(
+ new Function2<Integer, Integer, Integer>() {
+ @Override public Integer call(Integer i1, Integer i2) throws Exception {
+ return i1 + i2;
+ }
+ });
+wordCount.print(); // Print a few of the counts to the console
+{% endhighlight %}
+
+The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word,
+1)` pairs, using a [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction)
+object. Then, it is reduced to get the frequency of words in each batch of data,
+using a [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) object.
+Finally, `wordCounts.print()` will print a few of the counts generated every second.
+
+Note that when these lines are executed, Spark Streaming only sets up the computation it
+will perform when it is started, and no real processing has started yet. To start the processing
+after all the transformations have been setup, we finally call
+
+{% highlight java %}
+jssc.start(); // Start the computation
+jssc.awaitTermination(); // Wait for the computation to terminate
+{% endhighlight %}
+
+The complete code can be found in the Spark Streaming example
+[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java).
+<br>
+
+</div>
+</div>
+
+If you have already [downloaded](index.html#downloading) and [built](index.html#building) Spark,
+you can run this example as follows. You will first need to run Netcat
+(a small utility found in most Unix-like systems) as a data server by using
+
+{% highlight bash %}
+$ nc -lk 9999
+{% endhighlight %}
+
+Then, in a different terminal, you can start the example by using
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+{% highlight bash %}
+$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+{% endhighlight %}
+</div>
+<div data-lang="java" markdown="1">
+{% highlight bash %}
+$ ./bin/run-example org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999
+{% endhighlight %}
+</div>
+</div>
+
+
+Then, any lines typed in the terminal running the netcat server will be counted and printed on
+screen every second. It will look something like this.
+
+<table width="100%">
+ <td>
+{% highlight bash %}
+# TERMINAL 1:
+# Running Netcat
+
+$ nc -lk 9999
+
+hello world
+
+
+
+...
{% endhighlight %}
+ </td>
+ <td width="2%"></td>
+ <td>
+{% highlight bash %}
+# TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount
-where `conf` is a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
-object used for more advanced configuration. In both cases, a [SparkContext](api/core/index.html#org.apache.spark.SparkContext) is created as well which can be accessed with `streamingContext.sparkContext`.
+$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+...
+-------------------------------------------
+Time: 1357008430000 ms
+-------------------------------------------
+(hello,1)
+(world,1)
+...
+{% endhighlight %}
+ </td>
+</table>
+
+***************************************************************************************************
-# Attaching Input Sources
-The StreamingContext is used to creating input streams from data sources:
+# Basics
+
+Next, we move beyond the simple example and elaborate on the basics of Spark Streaming that you
+need to know to write your streaming applications.
+
+## Linking
+
+To write your own Spark Streaming program, you will have to add the following dependency to your
+ SBT or Maven project:
+
+ groupId = org.apache.spark
+ artifactId = spark-streaming_{{site.SCALA_VERSION}}
+ version = {{site.SPARK_VERSION}}
+
+For ingesting data from sources like Kafka and Flume that are not present in the Spark
+Streaming core
+ API, you will have to add the corresponding
+artifact `spark-streaming-xyz_{{site.SCALA_VERSION}}` to the dependencies. For example,
+some of the common ones are as follows.
+
+
+<table class="table">
+<tr><th>Source</th><th>Artifact</th></tr>
+<tr><td> Kafka </td><td> spark-streaming-kafka_{{site.SCALA_VERSION}} </td></tr>
+<tr><td> Flume </td><td> spark-streaming-flume_{{site.SCALA_VERSION}} </td></tr>
+<tr><td> Twitter </td><td> spark-streaming-twitter_{{site.SCALA_VERSION}} </td></tr>
+<tr><td> ZeroMQ </td><td> spark-streaming-zeromq_{{site.SCALA_VERSION}} </td></tr>
+<tr><td> MQTT </td><td> spark-streaming-mqtt_{{site.SCALA_VERSION}} </td></tr>
+<tr><td> </td><td></td></tr>
+</table>
+
+For an up-to-date list, please refer to the
+[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION}}%22)
+for the full list of supported sources and artifacts.
+
+## Initializing
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+To initialize a Spark Streaming program in Scala, a
+[`StreamingContext`](api/streaming/index.html#org.apache.spark.streaming.StreamingContext)
+object has to be created, which is the main entry point of all Spark Streaming functionality.
+A `StreamingContext` object can be created by using
{% highlight scala %}
-// Assuming ssc is the StreamingContext
-ssc.textFileStream(directory) // Creates a stream that monitors and processes new files in a HDFS directory
-ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port
+new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
{% endhighlight %}
+</div>
+<div data-lang="java" markdown="1">
+
+To initialize a Spark Streaming program in Java, a
+[`JavaStreamingContext`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext)
+object has to be created, which is the main entry point of all Spark Streaming functionality.
+A `JavaStreamingContext` object can be created by using
-The core Spark Streaming API provides input streams for files, sockets, and Akka actors. Additional functionality for Kafka, Flume, ZeroMQ, Twitter, etc. can be imported by adding the right dependencies as explained in the [linking](#linking-with-spark-streaming) section.
+{% highlight scala %}
+new JavaStreamingContext(master, appName, batchInterval, [sparkHome], [jars])
+{% endhighlight %}
+</div>
+</div>
+
+The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls)
+and can be "local" for local testing. The `appName` is a name of your program,
+which will be shown on your cluster's web UI. The `batchInterval` is the size of the batches,
+as explained earlier. Finally, the last two parameters are needed to deploy your code to a cluster
+ if running in distributed mode, as described in the
+ [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
+ Additionally, the underlying SparkContext can be accessed as
+`streamingContext.sparkContext`.
+
+The batch interval must be set based on the latency requirements of your application
+and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size)
+section for more details.
+
+## DStreams
+*Discretized Stream* or *DStream* is the basic abstraction provided by Spark Streaming.
+It represents a continuous stream of data, either the input data stream received from source,
+or the processed data stream generated by transforming the input stream. Internally,
+it is represented by a continuous sequence of RDDs, which is Spark's abstraction of an immutable,
+distributed dataset. Each RDD in a DStream contains data from a certain interval,
+as shown in the following figure.
+
+<p style="text-align: center;">
+ <img src="img/streaming-dstream.png"
+ title="Spark Streaming data flow"
+ alt="Spark Streaming"
+ width="70%" />
+</p>
+
+Any operation applied on a DStream translates to operations on the underlying RDDs. For example,
+in the [earlier example](#a-quick-example) of converting a stream of lines to words,
+the `flatmap` operation is applied on each RDD in the `lines` DStream to generate the RDDs of the
+ `words` DStream. This is shown the following figure.
+
+<p style="text-align: center;">
+ <img src="img/streaming-dstream-ops.png"
+ title="Spark Streaming data flow"
+ alt="Spark Streaming"
+ width="70%" />
+</p>
+
+
+These underlying RDD transformations are computed by the Spark engine. The DStream operations
+hide most of these details and provides the developer with higher-level API for convenience.
+These operations are discussed in detail in later sections.
+
+## Input Sources
+
+We have already taken a look at the `streamingContext.socketTextStream(...)` in the [quick
+example](#a-quick-example) which creates a DStream from text
+data received over a TCP socket connection. Besides sockets, the core Spark Streaming API provides
+methods for creating DStreams from files and Akka actors as input sources.
+
+Specifically, for files, the DStream can be created as
+
+<div class="codetabs">
+<div data-lang="scala">
+{% highlight scala %}
+streamingContext.fileStream(dataDirectory)
+{% endhighlight %}
+</div>
+<div data-lang="java">
+{% highlight java %}
+javaStreamingContext.fileStream(dataDirectory);
+{% endhighlight %}
+</div>
+</div>
+
+Spark Streaming will monitor the directory `dataDirectory` for any Hadoop-compatible filesystem
+and process any files created in that directory. Note that
+
+ * The files must have the same data format.
+ * The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into
+ the data directory.
+ * Once moved the files must not be changed.
+
+For more details on streams from files, Akka actors and sockets,
+see the API documentations of the relevant functions in
+[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) for
+Scala and [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext)
+ for Java.
+
+Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter
+can be imported by adding the right dependencies as explained in an
+[earlier](#linking) section. To take the
+case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_VERSION}}` to the
+project dependencies, you can create a DStream from Kafka as
+
+<div class="codetabs">
+<div data-lang="scala">
+{% highlight scala %}
+import org.apache.spark.streaming.kafka._
+KafkaUtils.createStream(streamingContext, kafkaParams, ...)
+{% endhighlight %}
+</div>
+<div data-lang="java">
+{% highlight java %}
+import org.apache.spark.streaming.kafka.*
+KafkaUtils.createStream(javaStreamingContext, kafkaParams, ...);
+{% endhighlight %}
+</div>
+</div>
-# 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, output operations need to called, which write data out to an external data sink like a file system or a database.
+For more details on these additional sources, see the corresponding [API documentation]
+(#where-to-go-from-here). Furthermore, you can also implement your own custom receiver
+for your sources. See the [Custom Receiver Guide](streaming-custom-receivers.html).
-## Transformations
+## Operations
+There are two kinds of DStream 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, output
+operations need to called, which write data out to an external data sink, such as a filesystem or a
+database.
-DStreams support many of the transformations available on normal Spark RDD's:
+### Transformations
+DStreams support many of the transformations available on normal Spark RDD's. Some of the
+common ones are as follows.
<table class="table">
-<tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
+<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
<tr>
<td> <b>map</b>(<i>func</i>) </td>
- <td> Returns a new DStream formed by passing each element of the source DStream through a function <i>func</i>. </td>
-</tr>
-<tr>
- <td> <b>filter</b>(<i>func</i>) </td>
- <td> Returns a new DStream formed by selecting those elements of the source DStream on which <i>func</i> returns true. </td>
+ <td> Return a new DStream by passing each element of the source DStream through a
+ function <i>func</i>. </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 <code>Seq</code> rather than a single item). </td>
+ <td> Similar to map, but each input item can be mapped to 0 or more output items. </td>
</tr>
<tr>
- <td> <b>mapPartitions</b>(<i>func</i>) </td>
- <td> Similar to map, but runs separately on each partition (block) of the DStream, so <i>func</i> must be of type
- Iterator[T] => Iterator[U] when running on an DStream of type T. </td>
+ <td> <b>filter</b>(<i>func</i>) </td>
+ <td> Return a new DStream by selecting only the records of the source DStream on which
+ <i>func</i> returns true. </td>
</tr>
<tr>
<td> <b>repartition</b>(<i>numPartitions</i>) </td>
@@ -82,329 +465,681 @@ 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 DStream that contains the union of the elements in the source DStream and the argument DStream. </td>
+ <td> Return a new DStream that contains the union of the elements in the source DStream and
+ <i>otherDStream</i>. </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>
+ <td> Return 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>
+ <td> Return 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 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 cluster) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
-</td>
+ <td> When called on a DStream of elements of type K, return 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>reduceByKey</b>(<i>func</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. 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, return a new DStream of (K, V) pairs where the
+ values for each key are aggregated using the given reduce function. <b>Note:</b> By default,
+ this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) 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>join</b>(<i>otherStream</i>, [<i>numTasks</i>]) </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>
+ <td> When called on two DStreams of (K, V) and (K, W) pairs, return 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 (K, V) and (K, W) pairs, returns a new DStream of (K, Seq[V], Seq[W]) tuples.</td>
+ <td> When called on DStream of (K, V) and (K, W) pairs, return 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>
+ <td> Return a new DStream by applying a RDD-to-RDD function to every RDD of the source DStream.
+ 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 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:30%">Transformation</th><th>Meaning</th></tr>
-<tr>
- <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>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> 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>(<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, 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 cluster) 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>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>
+ <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 for the key. This can be
+ used to maintain arbitrary state data for each ket.</td>
</tr>
+<tr><td></td><td></td></tr>
</table>
-A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#org.apache.spark.streaming.dstream.DStream) and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions).
+The last two transformations are worth highlighting again.
-## Output Operations
-When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined:
+<h4>UpdateStateByKey Operation</h4>
-<table class="table">
-<tr><th style="width:30%">Operator</th><th>Meaning</th></tr>
-<tr>
- <td> <b>foreachRDD</b>(<i>func</i>) </td>
- <td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td>
-</tr>
+The `updateStateByKey` operation allows
+you to main arbitrary stateful computation, where you want to maintain some state data and
+continuously update it with new information. To use this, you will have to do two steps.
-<tr>
- <td> <b>print</b>() </td>
- <td> Prints first ten elements of every batch of data in a DStream on the driver. </td>
-</tr>
+1. Define the state - The state can be of arbitrary data type.
+1. Define the state update function - Specify with a function how to update the state using the
+previous state and the new values from input stream.
-<tr>
- <td> <b>saveAsObjectFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
- <td> Save this DStream's contents as a <code>SequenceFile</code> of serialized objects. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>.
- </td>
-</tr>
+Let's illustrate this with an example. Say you want to maintain a running count of each word
+seen in a text data stream. Here, the running count is the state and it is an integer. We
+define the update function as
-<tr>
- <td> <b>saveAsTextFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
- <td> Save this DStream's contents as a text files. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
-</tr>
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
-<tr>
- <td> <b>saveAsHadoopFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
- <td> Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
-</tr>
+{% highlight scala %}
+def updateFunction(newValues: Seq[Int], runningCount: Option[Int]): Option[Int] = {
+ val newCount = ... // add the new values with the previous running count to get the new count
+ Some(newCount)
+}
+{% endhighlight %}
-</table>
+This is applied on a DStream containing words (say, the `pairs` DStream containing `(word,
+1)` pairs in the [earlier example](#a-quick-example)).
-# Starting the Streaming computation
-All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using
{% highlight scala %}
-ssc.start()
+val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
{% endhighlight %}
-Conversely, the computation can be stopped by using
-{% highlight scala %}
-ssc.stop()
+</div>
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+Function2<List<Integer>, Optional<Integer>, Optional<Integer>> updateFunction =
+ new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
+ @Override public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
+ Integer newSum = ... // add the new values with the previous running count to get the new count
+ return Optional.of(newSum)
+ }
+ }
{% endhighlight %}
-# Example
-A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/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/org/apache/spark/streaming/examples/NetworkWordCount.scala` .
+This is applied on a DStream containing words (say, the `pairs` DStream containing `(word,
+1)` pairs in the [quick example](#a-quick-example)).
-{% highlight scala %}
-import org.apache.spark.streaming.{Seconds, StreamingContext}
-import StreamingContext._
-...
+{% highlight java %}
+JavaPairDStream<String, Integer> runningCounts = pairs.updateStateByKey(updateFunction);
+{% endhighlight %}
-// 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.socketTextStream(args(1), args(2).toInt)
+</div>
+</div>
-// Split the lines into words, count them, and print some of the counts on the master
-val words = lines.flatMap(_.split(" "))
-val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
-wordCounts.print()
+The update function will be called for each word, with `newValues` having a sequence of 1's (from
+the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
+Scala code, take a look at the example
+[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala).
-// Start the computation
-ssc.start()
-{% endhighlight %}
+<h4>Transform Operation</h4>
-The `socketTextStream` returns a DStream of text data received from a TCP server socket. 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.
+The `transform` operation (along with its variations like `transformWith`) allows
+arbitrary RDD-to-RDD functions to be applied on a DStream. It can be used to apply any RDD
+operation that is not exposed in the DStream API.
+For example, the functionality of joining every batch in a data stream
+with another dataset is not directly exposed in the DStream API. However,
+you can easily use `transform` to do this. This enables very powerful possibilities. For example,
+if you want to do real-time data cleaning by joining the input data stream with precomputed
+spam information (maybe generated with Spark as well) and then filtering based on it.
-To run this example on your local machine, you need to first run a Netcat server by using
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
-{% highlight bash %}
-$ nc -lk 9999
+{% highlight scala %}
+val spamInfoRDD = sparkContext.hadoopFile(...) // RDD containing spam information
+
+val cleanedDStream = inputDStream.transform(rdd => {
+ rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning
+ ...
+})
{% endhighlight %}
-Then, in a different terminal, you can start NetworkWordCount by using
+</div>
+<div data-lang="java" markdown="1">
-{% highlight bash %}
-$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+{% highlight java %}
+// RDD containing spam information
+JavaPairRDD<String, Double> spamInfoRDD = javaSparkContext.hadoopFile(...);
+
+JavaPairDStream<String, Integer> cleanedDStream = inputDStream.transform(
+ new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>>() {
+ @Override public JavaPairRDD<String, Integer> call(JavaPairRDD<String, Integer> rdd) throws Exception {
+ rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning
+ ...
+ }
+ });
{% endhighlight %}
-This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
+</div>
+</div>
-<table>
-<td>
-{% highlight bash %}
-# TERMINAL 1
-# RUNNING NETCAT
+In fact, you can also use [machine learning](mllib-guide.html) and
+[graph computation](graphx-programming-guide.html) algorithms in the `transform` method.
-$ nc -lk 9999
-hello world
+<h4>Window Operations</h4>
+Finally, Spark Streaming also provides *windowed computations*, which allow you to apply
+transformations over a sliding window of data. This following figure illustrates this sliding
+window.
+<p style="text-align: center;">
+ <img src="img/streaming-dstream-window.png"
+ title="Spark Streaming data flow"
+ alt="Spark Streaming"
+ width="60%" />
+</p>
+As shown in the figure, every time the window *slides* over a source DStream,
+the source RDDs that fall within the window are combined and operated upon to produce the
+RDDs of the windowed DStream. In this specific case, the operation is applied over last 3 time
+units of data, and slides by 2 time units. This shows that any window-based operation needs to
+specify two parameters.
+ * <i>window length</i> - The duration of the window (3 in the figure)
+ * <i>slide interval</i> - The interval at which the window-based operation is performed (2 in
+ the figure).
-...
+These two parameters must be multiples of the batch interval of the source DStream (1 in the
+figure).
+
+Let's illustrate the window operations with an example. Say, you want to extend the
+[earlier example](#a-quick-example) by generating word counts over last 30 seconds of data,
+every 10 seconds. To do this, we have to apply the `reduceByKey` operation on the `pairs` DStream of
+`(word, 1)` pairs over the last 30 seconds of data. This is done using the
+operation `reduceByKeyAndWindow`.
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+// Reduce last 30 seconds of data, every 10 seconds
+val windowedWordCounts = pairs.reduceByKeyAndWindow(_ + _, Seconds(30), Seconds(10))
{% endhighlight %}
-</td>
-<td>
-{% highlight bash %}
-# TERMINAL 2: RUNNING NetworkWordCount
-...
--------------------------------------------
-Time: 1357008430000 ms
--------------------------------------------
-(hello,1)
-(world,1)
-...
+</div>
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+// Reduce function adding two integers, defined separately for clarity
+Function2<Integer, Integer, Integer> reduceFunc = new Function2<Integer, Integer, Integer>() {
+ @Override public Integer call(Integer i1, Integer i2) throws Exception {
+ return i1 + i2;
+ }
+};
+
+// Reduce last 30 seconds of data, every 10 seconds
+JavaPairDStream<String, Integer> windowedWordCounts = pair.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000));
{% endhighlight %}
-</td>
-</table>
-You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+</div>
+</div>
-# 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()`.
+Some of the common window-based operations are as follows. All of these operations take the
+said two parameters - <i>windowLength</i> and <i>slideInterval</i>.
-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.
+<table class="table">
+<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
+<tr>
+ <td> <b>window</b>(<i>windowLength</i>, <i>slideInterval</i>) </td>
+ <td> Return a new DStream which is computed based on windowed batches of the source DStream.
+ </td>
+</tr>
+<tr>
+ <td> <b>countByWindow</b>(<i>windowLength</i>, <i>slideInterval</i>) </td>
+ <td> Return a sliding window count of elements in the stream.
+ </td>
+</tr>
+<tr>
+ <td> <b>reduceByWindow</b>(<i>func</i>, <i>windowLength</i>, <i>slideInterval</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.
+ </td>
+</tr>
+<tr>
+ <td> <b>reduceByKeyAndWindow</b>(<i>func</i>, <i>windowLength</i>, <i>slideInterval</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. <b>Note:</b> By default, this uses Spark's default number of
+ parallel tasks (2 for local machine, 8 for a cluster) 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>invFunc</i>, <i>windowLength</i>,
+ <i>slideInterval</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>reduceByKeyAndWindow</code>, the number of reduce tasks
+ is configurable through an optional argument.
+</td>
+</tr>
+<tr>
+ <td> <b>countByValueAndWindow</b>(<i>windowLength</i>,
+ <i>slideInterval</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>reduceByKeyAndWindow</code>, the number of reduce tasks is configurable through an
+ optional argument.
+</td>
+</tr>
+<tr><td></td><td></td></tr>
+</table>
-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).
+### Output Operations
+When an output operator is called, it triggers the computation of a stream. Currently the following
+output operators are defined:
-# RDD Checkpointing within DStreams
-A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation.
+<table class="table">
+<tr><th style="width:30%">Output Operation</th><th>Meaning</th></tr>
+<tr>
+ <td> <b>print</b>() </td>
+ <td> Prints first ten elements of every batch of data in a DStream on the driver. </td>
+</tr>
+<tr>
+ <td> <b>foreachRDD</b>(<i>func</i>) </td>
+ <td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from
+ the stream. This function should have side effects, such as printing output, saving the RDD to
+ external files, or writing it over the network to an external system. </td>
+</tr>
+<tr>
+ <td> <b>saveAsObjectFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a <code>SequenceFile</code> of serialized objects. The file
+ name at each batch interval is generated based on <i>prefix</i> and
+ <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>.
+ </td>
+</tr>
+<tr>
+ <td> <b>saveAsTextFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a text files. The file name at each batch interval is
+ generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+<tr>
+ <td> <b>saveAsHadoopFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+ <td> Save this DStream's contents as a Hadoop file. The file name at each batch interval is
+ generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+<tr><td></td><td></td></tr>
+</table>
-Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this metadata, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try.
-To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using
+The complete list of DStream operations is available in the API documentation. For the Scala API,
+see [DStream](api/streaming/index.html#org.apache.spark.streaming.dstream.DStream)
+and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions).
+For the Java API, see [JavaDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.dstream.DStream)
+and [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream).
+Specifically for the Java API, see [Spark's Java programming guide](java-programming-guide.html)
+for more information.
+
+## 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 over the network (such as, Kafka, Flume, sockets, etc.), 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
+A _stateful operation_ is one which operates over multiple batches of data. This includes all
+window-based operations and the `updateStateByKey` operation. Since stateful operations have a
+dependency on previous batches of data, they continuously accumulate metadata over time.
+To clear this metadata, streaming supports periodic _checkpointing_ by saving intermediate data
+to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the
+corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be
+set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly
+reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task
+sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10
+times of sliding interval of a DStream is good setting to try.
+
+To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved.
+This is done by using
{% highlight scala %}
-ssc.checkpoint(hdfsPath) // assuming ssc is the StreamingContext
+ssc.checkpoint(hdfsPath) // assuming ssc is the StreamingContext or JavaStreamingContext
{% 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
+dstream.checkpoint(checkpointInterval)
{% 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.
-
+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.
-## Custom Receivers
-Spark comes with a built in support for most common usage scenarios where input stream source can be either a network socket stream to support for a few message queues. Apart from that it is also possible to supply your own custom receiver via a convenient API. Find more details at [Custom Receiver Guide](streaming-custom-receivers.html).
+***************************************************************************************************
# Performance Tuning
-Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
+Getting the best performance of a Spark Streaming application on a cluster requires a bit of
+tuning. This section explains a number of the parameters and configurations that can tuned to
+improve the performance of you application. At a high level, you need to consider two things:
+
<ol>
-<li>Reducing the processing time of each batch of data by efficiently using cluster resources.</li>
-<li>Setting the right batch size such that the data processing can keep up with the data ingestion.</li>
+<li>
+ Reducing the processing time of each batch of data by efficiently using cluster resources.
+</li>
+<li>
+ Setting the right batch size such that the data processing can keep up with the data ingestion.
+</li>
</ol>
## Reducing the Processing Time of each Batch
-There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
+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 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 [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `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
+[`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions)
+documentation), or set the [config property](configuration.html#spark-properties)
+`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.
+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 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 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:
+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.
+* **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.
+* **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.
-## Setting the Right Batch Size
-For a Spark Streaming application running on a cluster to be stable, the processing of the data streams must keep up with the rate of ingestion of the data streams. Depending on the type of computation, the batch size used may have significant impact on the rate of ingestion that can be sustained by the Spark Streaming application on a fixed cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not every 500 milliseconds.
+These changes may reduce batch processing time by 100s of milliseconds,
+thus allowing sub-second batch size to be viable.
-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).
+## Setting the Right Batch Size
+For a Spark Streaming application running on a cluster to be stable, the processing of the data
+streams must keep up with the rate of ingestion of the data streams. Depending on the type of
+computation, the batch size used may have significant impact on the rate of ingestion that can be
+sustained by the Spark Streaming application on a fixed cluster resources. For example, let us
+consider the earlier WordCountNetwork example. For a particular data rate, the system may be able
+to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not
+every 500 milliseconds.
+
+A good approach to figure out the right batch size for your application is to test it with a
+conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system
+is able to keep up with data rate, you can check the value of the end-to-end delay experienced
+by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the
+[StreamingListener](api/streaming/index.html#org.apache.spark.streaming.scheduler.StreamingListener)
+interface).
+If the delay is maintained to be comparable to 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 [config property](configuration.html#spark-properties) `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.
+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
+[configuration property](configuration.html#spark-properties) `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.
+
+## Monitoring
+Besides Spark's in-built [monitoring capabilities](monitoring.html),
+the progress of a Spark Streaming program can also be monitored using the [StreamingListener]
+(streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface,
+which allows you to get statistics of batch processing times, queueing delays,
+and total end-to-end delays. Note that this is still an experimental API and it is likely to be
+improved upon (i.e., more information reported) in the future.
## 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.
-
-* **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#org.apache.spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses.
-
-* **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.
+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.
+
+* **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#org.apache.spark.storage.StorageLevel$) for
+DStream compared to
+[StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs).
+Even though keeping the data serialized incurs higher serialization/deserialization overheads,
+it significantly reduces GC pauses.
+
+* **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will
+ be cleared from memory based on Spark's in-built policy (LRU). If `spark.cleaner.ttl` is set,
+ then persistent RDDs that are older than that value are periodically cleared. As mentioned
+ [earlier](#operation), this needs to be careful set based on operations used in the Spark
+ Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the
+ [configuration property](configuration.html#spark-properties) `spark.streaming.unpersist` to
+ `true`. This makes the system to figure out which RDDs are not necessary to be kept around and
+ unpersists them. This is likely to reduce
+ the RDD memory usage of Spark, potentially improving GC behavior as well.
+
+* **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.
+
+***************************************************************************************************
# 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.
+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.
- 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.
+ 1. An RDD is an immutable, 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.
-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.
+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.
## 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 `foreachRDD`) 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.
+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 `foreachRDD`) 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.
+To allows a streaming application to operate 24/7, Spark Streaming allows a streaming computation
+to be resumed even after the failure of the driver node. Spark Streaming periodically writes the
+metadata information of the DStreams setup through the `StreamingContext` to a
+HDFS directory (can be any Hadoop-compatible filesystem). This periodic
+*checkpointing* can be enabled by setting a the checkpoint
+directory using `ssc.checkpoint(<checkpoint directory>)` as described
+[earlier](#rdd-checkpointing). On failure of the driver node,
+the lost `StreamingContext` can be recovered from this information, and restarted.
+
+To allow a Spark Streaming program to be recoverable, it must be written in a way such that
+it has the following behavior:
-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
+1. When the program is being started for the first time, it will create a new StreamingContext,
+ set up all the streams and then call start().
+1. When the program is being restarted after failure, it will re-create a StreamingContext
+ from the checkpoint data in the checkpoint directory.
-All this is periodically saved in the checkpoint directory. To recover, a new `StreamingContext` can be created with this directory by using
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+This behavior is made simple by using `StreamingContext.getOrCreate`. This is used as follows.
{% highlight scala %}
-val ssc = new StreamingContext(checkpointDirectory)
+// Function to create and setup a new StreamingContext
+def functionToCreateContext(): StreamingContext = {
+ val ssc = new StreamingContext(...) // new context
+ val lines = ssc.socketTextStream(...) // create DStreams
+ ...
+ ssc.checkpoint(checkpointDirectory) // set checkpoint directory
+ ssc
+}
+
+// Get StreaminContext from checkpoint data or create a new one
+val context = StreamingContext.getOrCreate(checkpointDirectory, functionToCreateContext _)
+
+// Do additional setup on context that needs to be done,
+// irrespective of whether it is being started or restarted
+context. ...
+
+// Start the context
+context.start()
+context.awaitTermination()
{% endhighlight %}
-On calling `ssc.start()` on this new context, the following steps are taken by the system
+If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data.
+If the directory does not exist (i.e., running for the first time),
+then the function `functionToCreateContext` will be called to create a new
+context and set up the DStreams. See the Scala example
+[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala).
+This example appends the word counts of network data into a file.
-1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it last checkpointed. This is also done for those time steps that were previously scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc.
-1. Restart the network receivers, if any, and continue receiving new data.
+You can also explicitly create a `StreamingContext` from the checkpoint data and start the
+ computation by using `new StreamingContext(checkpointDirectory)`.
-There are two different failure behaviors based on which input sources are used.
+</div>
+<div data-lang="java" markdown="1">
+
+This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This is used as follows.
+
+{% highlight java %}
+// Create a factory object that can create a and setup a new JavaStreamingContext
+JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() {
+ JavaStreamingContextFactory create() {
+ JavaStreamingContext jssc = new JavaStreamingContext(...); // new context
+ JavaDStream<String> lines = jssc.socketTextStream(...); // create DStreams
+ ...
+ jssc.checkpoint(checkpointDirectory); // set checkpoint directory
+ return jssc;
+ }
+};
+
+// Get JavaStreamingContext from checkpoint data or create a new one
+JavaStreamingContext context = JavaStreamingContext.getOrCreate(checkpointDirectory, contextFactory);
+
+// Do additional setup on context that needs to be done,
+// irrespective of whether it is being started or restarted
+context. ...
+
+// Start the context
+context.start();
+context.awaitTermination();
+{% endhighlight %}
-1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
-1. _Using any input source that receives data through a network_ - The received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely.
+If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data.
+If the directory does not exist (i.e., running for the first time),
+then the function `contextFactory` will be called to create a new
+context and set up the DStreams. See the Scala example
+[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/JavaRecoverableWordCount.scala)
+(note that this example is missing in the 0.9 release, so you can test it using the master branch).
+This example appends the word counts of network data into a file.
+
+You can also explicitly create a `JavaStreamingContext` from the checkpoint data and start
+the computation by using `new JavaStreamingContext(checkpointDirectory)`.
+
+</div>
+</div>
+
+**Note**: If Spark Streaming and/or the Spark Streaming program is recompiled,
+you *must* create a new `StreamingContext` or `JavaStreamingContext`,
+not recreate from checkpoint data. This is because trying to load a
+context from checkpoint data may fail if the data was generated before recompilation of the
+classes. So, if you are using `getOrCreate`, then make sure that the checkpoint directory is
+explicitly deleted every time recompiled code needs to be launched.
+
+This failure recovery can be done automatically using Spark's
+[standalone cluster mode](spark-standalone.html), which allows any Spark
+application's driver to be as well as, ensures automatic restart of the driver on failure (see
+[supervise mode](spark-standalone.html#launching-applications-inside-the-cluster)). This can be
+tested locally by launching the above example using the supervise mode in a
+local standalone cluster and killing the java process running the driver (will be shown as
+*DriverWrapper* when `jps` is run to show all active Java processes). The driver should be
+automatically restarted, and the word counts will cont
+
+For other deployment environments like Mesos and Yarn, you have to restart the driver through other
+mechanisms.
+
+<h4>Recovery Semantics</h4>
-In future releases, we will support full recoverability for all input sources. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data.
+There are two different failure behaviors based on which input sources are used.
-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.
+1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can
+re-computed and therefore no data will be lost due to any failure.
+1. _Using any input source that receives data through a network_ - The received input data is
+replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost
+when the Spark driver fails, the past input data will not be accessible and driver recovers.
+Hence, if stateful and window-based operations are used
+(like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state
+will not be recovered completely.
+
+In future releases, we will support full recoverability for all input sources. Note that for
+non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams,
+the system, upon restarting, will continue to receive and process new data.
+
+To better understand the behavior of the system under driver failure with a HDFS source, lets
+consider what will happen with a file input stream. Specifically, in the case of the file input
+stream, it will correctly identify new files that were created while the driver was down and
+process them in the same way as it would have if the driver had not failed. To explain further
+in the case of file input stream, we shall use an example. Lets say, files are being generated
+every second, and a Spark Streaming program reads every new file and output the number of lines
+in the file. This is what the sequence of outputs would be with and without a driver failure.
<table class="table">
<!-- Results table headers -->
@@ -476,58 +1211,21 @@ To better understand the behavior of the system under driver failure with a HDFS
</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 [org.apache.spark.streaming.api.java] (api/streaming/index.html#org.apache.spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#org.apache.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#org.apache.spark.api.java.function.Function) and [Function2](api/core/index.html#org.apache.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#org.apache.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/org/apache/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#org.apache.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#org.apache.spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction). This is reduced by using `reduceByKey` and [Function2](api/core/index.html#org.apache.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 %}
-
+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.
# Where to Go from Here
-* API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package)
-* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/streaming/examples)
-* [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
+* API documentation
+ - Main docs of StreamingContext and DStreams in [Scala](api/streaming/index.html#org.apache.spark.streaming.package)
+ and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package)
+ - Additional docs for
+ [Kafka](api/external/kafka/index.html#org.apache.spark.streaming.kafka.KafkaUtils$),
+ [Flume](api/external/flume/index.html#org.apache.spark.streaming.flume.FlumeUtils$),
+ [Twitter](api/external/twitter/index.html#org.apache.spark.streaming.twitter.TwitterUtils$),
+ [ZeroMQ](api/external/zeromq/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and
+ [MQTT](api/external/mqtt/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$)
+
+* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples)
+ and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples)
+* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming