-
Shivansh authored
## What changes were proposed in this pull request? Fix the broken links in the programming guide of the Graphx Migration and understanding closures ## How was this patch tested? By running the test cases and checking the links. Author: Shivansh <shiv4nsh@gmail.com> Closes #14503 from shiv4nsh/SPARK-16911.
Shivansh authored## What changes were proposed in this pull request? Fix the broken links in the programming guide of the Graphx Migration and understanding closures ## How was this patch tested? By running the test cases and checking the links. Author: Shivansh <shiv4nsh@gmail.com> Closes #14503 from shiv4nsh/SPARK-16911.
layout: global
displayTitle: Spark Streaming Programming Guide
title: Spark Streaming
description: Spark Streaming programming guide and tutorial for Spark SPARK_VERSION_SHORT
- This will become a table of contents (this text will be scraped). {:toc}
Overview
Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput,
fault-tolerant stream processing of live data streams. Data can be ingested from many sources
like Kafka, Flume, Kinesis, or TCP sockets, and can 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
machine learning and
graph processing algorithms on data streams.
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.
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 streams from sources such as Kafka, Flume, and Kinesis, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of RDDs.
This guide shows you how to start writing Spark Streaming programs with DStreams. You can write Spark Streaming programs in Scala, Java or Python (introduced in Spark 1.2), all of which are presented in this guide. You will find tabs throughout this guide that let you choose between code snippets of different languages.
Note: There are a few APIs that are either different or not available in Python. Throughout this guide, you will find the tag Python API highlighting these differences.
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.
{% highlight scala %} import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3
// Create a local StreamingContext with two working thread and batch interval of 1 second. // The master requires 2 cores to prevent from a starvation scenario.
val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %}
Using this context, we can create a DStream that represents streaming data from a TCP
source, specified as hostname (e.g. localhost
) and port (e.g. 9999
).
{% highlight scala %} // Create a DStream that will connect to hostname:port, like localhost:9999 val lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %}
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 characters into words.
{% highlight scala %} // Split each line into words val words = lines.flatMap(_.split(" ")) {% endhighlight %}
flatMap
is a one-to-many 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. Next, we want to count these words.
{% highlight scala %} import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3 // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _)
// Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print() {% endhighlight %}
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 %} 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.
First, we create a JavaStreamingContext object, which is the main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and a batch interval of 1 second.
{% highlight java %} import org.apache.spark.; import org.apache.spark.api.java.function.; import org.apache.spark.streaming.; import org.apache.spark.streaming.api.java.; import scala.Tuple2;
// Create a local StreamingContext with two working thread and batch interval of 1 second SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount"); JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)); {% endhighlight %}
Using this context, we can create a DStream that represents streaming data from a TCP
source, specified as hostname (e.g. localhost
) and port (e.g. 9999
).
{% highlight java %} // Create a DStream that will connect to hostname:port, like localhost:9999 JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999); {% 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 lines by
space into words.
{% highlight java %} // Split each line into words JavaDStream words = lines.flatMap( new FlatMapFunction<String, String>() { @Override public Iterator call(String x) { return Arrays.asList(x.split(" ")).iterator(); } }); {% 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 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.mapToPair( new PairFunction<String, String, Integer>() { @Override public Tuple2<String, Integer> call(String s) { return new Tuple2<>(s, 1); } }); JavaPairDStream<String, Integer> wordCounts = pairs.reduceByKey( new Function2<Integer, Integer, Integer>() { @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } });
// Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print(); {% endhighlight %}
The words
DStream is further mapped (one-to-one transformation) to a DStream of (word, 1)
pairs, using a PairFunction
object. Then, it is reduced to get the frequency of words in each batch of data,
using a 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 after it is started, and no real processing has started yet. To start the processing
after all the transformations have been setup, we finally call start
method.
{% 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.
{% highlight python %} from pyspark import SparkContext from pyspark.streaming import StreamingContext
Create a local StreamingContext with two working thread and batch interval of 1 second
sc = SparkContext("local[2]", "NetworkWordCount") ssc = StreamingContext(sc, 1) {% endhighlight %}
Using this context, we can create a DStream that represents streaming data from a TCP
source, specified as hostname (e.g. localhost
) and port (e.g. 9999
).
{% highlight python %}
Create a DStream that will connect to hostname:port, like localhost:9999
lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %}
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.
{% highlight python %}
Split each line into words
words = lines.flatMap(lambda line: line.split(" ")) {% endhighlight %}
flatMap
is a one-to-many 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. Next, we want to count these words.
{% highlight python %}
Count each word in each batch
pairs = words.map(lambda word: (word, 1)) wordCounts = pairs.reduceByKey(lambda x, y: x + y)
Print the first ten elements of each RDD generated in this DStream to the console
wordCounts.pprint() {% endhighlight %}
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.pprint()
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 python %} 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.
If you have already downloaded and built 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
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 the following.
{% highlight bash %}
# TERMINAL 1:
# Running Netcat
$ nc -lk 9999 hello world ... {% endhighlight %} |
{% highlight bash %}
# TERMINAL 2: RUNNING NetworkWordCount
$ ./bin/run-example streaming.NetworkWordCount localhost 9999 ...Time: 1357008430000 ms(hello,1) (world,1) ... {% endhighlight %}
{% highlight bash %}
# TERMINAL 2: RUNNING JavaNetworkWordCount
$ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 ...Time: 1357008430000 ms(hello,1) (world,1) ... {% endhighlight %}
{% highlight bash %}
# TERMINAL 2: RUNNING network_wordcount.py
$ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 ...Time: 2014-10-14 15:25:21(hello,1) (world,1) ... {% endhighlight %} |
Basic Concepts
Next, we move beyond the simple example and elaborate on the basics of Spark Streaming.
Linking
Similar to Spark, Spark Streaming is available through Maven Central. To write your own Spark Streaming program, you will have to add the following dependency to your SBT or Maven project.
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_{{site.SCALA_BINARY_VERSION}}</artifactId>
<version>{{site.SPARK_VERSION}}</version>
</dependency>
libraryDependencies += "org.apache.spark" % "spark-streaming_{{site.SCALA_BINARY_VERSION}}" % "{{site.SPARK_VERSION}}"
For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark
Streaming core
API, you will have to add the corresponding
artifact spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}
to the dependencies. For example,
some of the common ones are as follows.
Source | Artifact |
---|---|
Kafka | spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} |
Flume | spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} |
Kinesis |
spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License] |
For an up-to-date list, please refer to the Maven repository for the full list of supported sources and artifacts.
Initializing StreamingContext
To initialize a Spark Streaming program, a StreamingContext object has to be created which is the main entry point of all Spark Streaming functionality.
A StreamingContext object can be created from a SparkConf object.
{% highlight scala %} import org.apache.spark._ import org.apache.spark.streaming._
val conf = new SparkConf().setAppName(appName).setMaster(master) val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %}
The appName
parameter is a name for your application to show on the cluster UI.
master
is a Spark, Mesos or YARN cluster URL,
or a special "local[*]" string to run in local mode. In practice, when running on a cluster,
you will not want to hardcode master
in the program,
but rather launch the application with spark-submit
and
receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming
in-process (detects the number of cores in the local system). Note that this internally creates a SparkContext (starting point of all Spark functionality) which can be accessed as ssc.sparkContext
.
The batch interval must be set based on the latency requirements of your application and available cluster resources. See the Performance Tuning section for more details.
A StreamingContext
object can also be created from an existing SparkContext
object.
{% highlight scala %} import org.apache.spark.streaming._
val sc = ... // existing SparkContext val ssc = new StreamingContext(sc, Seconds(1)) {% endhighlight %}
A JavaStreamingContext object can be created from a SparkConf object.
{% highlight java %} import org.apache.spark.; import org.apache.spark.streaming.api.java.;
SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(1000)); {% endhighlight %}
The appName
parameter is a name for your application to show on the cluster UI.
master
is a Spark, Mesos or YARN cluster URL,
or a special "local[*]" string to run in local mode. In practice, when running on a cluster,
you will not want to hardcode master
in the program,
but rather launch the application with spark-submit
and
receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming
in-process. Note that this internally creates a JavaSparkContext (starting point of all Spark functionality) which can be accessed as ssc.sparkContext
.
The batch interval must be set based on the latency requirements of your application and available cluster resources. See the Performance Tuning section for more details.
A JavaStreamingContext
object can also be created from an existing JavaSparkContext
.
{% highlight java %} import org.apache.spark.streaming.api.java.*;
JavaSparkContext sc = ... //existing JavaSparkContext JavaStreamingContext ssc = new JavaStreamingContext(sc, Durations.seconds(1)); {% endhighlight %}
A StreamingContext object can be created from a SparkContext object.
{% highlight python %} from pyspark import SparkContext from pyspark.streaming import StreamingContext
sc = SparkContext(master, appName) ssc = StreamingContext(sc, 1) {% endhighlight %}
The appName
parameter is a name for your application to show on the cluster UI.
master
is a Spark, Mesos or YARN cluster URL,
or a special "local[*]" string to run in local mode. In practice, when running on a cluster,
you will not want to hardcode master
in the program,
but rather launch the application with spark-submit
and
receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming
in-process (detects the number of cores in the local system).
The batch interval must be set based on the latency requirements of your application and available cluster resources. See the Performance Tuning section for more details.
After a context is defined, you have to do the following.
- Define the input sources by creating input DStreams.
- Define the streaming computations by applying transformation and output operations to DStreams.
- Start receiving data and processing it using
streamingContext.start()
. - Wait for the processing to be stopped (manually or due to any error) using
streamingContext.awaitTermination()
. - The processing can be manually stopped using
streamingContext.stop()
.
Points to remember:
{:.no_toc}
- Once a context has been started, no new streaming computations can be set up or added to it.
- Once a context has been stopped, it cannot be restarted.
- Only one StreamingContext can be active in a JVM at the same time.
- stop() on StreamingContext also stops the SparkContext. To stop only the StreamingContext, set the optional parameter of
stop()
calledstopSparkContext
to false. - A SparkContext can be re-used to create multiple StreamingContexts, as long as the previous StreamingContext is stopped (without stopping the SparkContext) before the next StreamingContext is created.
Discretized Streams (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, a DStream is represented by a continuous series of RDDs, which is Spark's abstraction of an immutable, distributed dataset (see Spark Programming Guide for more details). Each RDD in a DStream contains data from a certain interval, as shown in the following figure.
Any operation applied on a DStream translates to operations on the underlying RDDs. For example,
in the earlier 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 in the following figure.
These underlying RDD transformations are computed by the Spark engine. The DStream operations hide most of these details and provide the developer with a higher-level API for convenience. These operations are discussed in detail in later sections.
Input DStreams and Receivers
Input DStreams are DStreams representing the stream of input data received from streaming
sources. In the quick example, lines
was an input DStream as it represented
the stream of data received from the netcat server. Every input DStream
(except file stream, discussed later in this section) is associated with a Receiver
(Scala doc,
Java doc) object which receives the
data from a source and stores it in Spark's memory for processing.
Spark Streaming provides two categories of built-in streaming sources.
- Basic sources: Sources directly available in the StreamingContext API. Examples: file systems, and socket connections.
- Advanced sources: Sources like Kafka, Flume, Kinesis, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the linking section.
We are going to discuss some of the sources present in each category later in this section.
Note that, if you want to receive multiple streams of data in parallel in your streaming application, you can create multiple input DStreams (discussed further in the Performance Tuning section). This will create multiple receivers which will simultaneously receive multiple data streams. But note that a Spark worker/executor is a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Therefore, it is important to remember that a Spark Streaming application needs to be allocated enough cores (or threads, if running locally) to process the received data, as well as to run the receiver(s).
Points to remember
{:.no_toc}
-
When running a Spark Streaming program locally, do not use "local" or "local[1]" as the master URL. Either of these means that only one thread will be used for running tasks locally. If you are using an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when running locally, always use "local[n]" as the master URL, where n > number of receivers to run (see Spark Properties for information on how to set the master).
-
Extending the logic to running on a cluster, the number of cores allocated to the Spark Streaming application must be more than the number of receivers. Otherwise the system will receive data, but not be able to process it.
Basic Sources
{:.no_toc}
We have already taken a look at the ssc.socketTextStream(...)
in the quick example
which creates a DStream from text
data received over a TCP socket connection. Besides sockets, the StreamingContext API provides
methods for creating DStreams from files as input sources.
-
File Streams: For reading data from files on any file system compatible with the HDFS API (that is, HDFS, S3, NFS, etc.), a DStream can be created as:
streamingContext.fileStreamKeyClass, ValueClass, InputFormatClassstreamingContext.fileStream(dataDirectory);streamingContext.textFileStream(dataDirectory)Spark Streaming will monitor the directory
dataDirectory
and process any files created in that directory (files written in nested directories not supported). 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. So if the files are being continuously appended, the new data will not be read.
For simple text files, there is an easier method
streamingContext.textFileStream(dataDirectory)
. And file streams do not require running a receiver, hence does not require allocating cores.Python API
fileStream
is not available in the Python API, onlytextFileStream
is available. -
Streams based on Custom Receivers: DStreams can be created with data streams received through custom receivers. See the Custom Receiver Guide and DStream Akka for more details.
-
Queue of RDDs as a Stream: For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using
streamingContext.queueStream(queueOfRDDs)
. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream.
For more details on streams from sockets and files, see the API documentations of the relevant functions in StreamingContext for Scala, JavaStreamingContext for Java, and StreamingContext for Python.
Advanced Sources
{:.no_toc}
Python API As of Spark {{site.SPARK_VERSION_SHORT}}, out of these sources, Kafka, Kinesis and Flume are available in the Python API.
This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate libraries that can be linked to explicitly when necessary.
Note that these advanced sources are not available in the Spark shell, hence applications based on these advanced sources cannot be tested in the shell. If you really want to use them in the Spark shell you will have to download the corresponding Maven artifact's JAR along with its dependencies and add it to the classpath.
Some of these advanced sources are as follows.
-
Kafka: Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the Kafka Integration Guide for more details.
-
Flume: Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.6.0. See the Flume Integration Guide for more details.
-
Kinesis: Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kinesis Client Library 1.2.1. See the Kinesis Integration Guide for more details.
Custom Sources
{:.no_toc}
Python API This is not yet supported in Python.
Input DStreams can also be created out of custom data sources. All you have to do is implement a user-defined receiver (see next section to understand what that is) that can receive data from the custom sources and push it into Spark. See the Custom Receiver Guide for details.
Receiver Reliability
{:.no_toc}
There can be two kinds of data sources based on their reliability. Sources (like Kafka and Flume) allow the transferred data to be acknowledged. If the system receiving data from these reliable sources acknowledges the received data correctly, it can be ensured that no data will be lost due to any kind of failure. This leads to two kinds of receivers:
- Reliable Receiver - A reliable receiver correctly sends acknowledgment to a reliable source when the data has been received and stored in Spark with replication.
- Unreliable Receiver - An unreliable receiver does not send acknowledgment to a source. This can be used for sources that do not support acknowledgment, or even for reliable sources when one does not want or need to go into the complexity of acknowledgment.
The details of how to write a reliable receiver are discussed in the Custom Receiver Guide.
Transformations on DStreams
Similar to that of RDDs, transformations allow the data from the input DStream to be modified. DStreams support many of the transformations available on normal Spark RDD's. Some of the common ones are as follows.
Transformation | Meaning |
---|---|
map(func) | Return a new DStream by passing each element of the source DStream through a function func. |
flatMap(func) | Similar to map, but each input item can be mapped to 0 or more output items. |
filter(func) | Return a new DStream by selecting only the records of the source DStream on which func returns true. |
repartition(numPartitions) | Changes the level of parallelism in this DStream by creating more or fewer partitions. |
union(otherStream) | Return a new DStream that contains the union of the elements in the source DStream and otherDStream. |
count() | Return a new DStream of single-element RDDs by counting the number of elements in each RDD of the source DStream. |
reduce(func) | Return a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function func (which takes two arguments and returns one). The function should be associative and commutative so that it can be computed in parallel. |
countByValue() | 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. |
reduceByKey(func, [numTasks]) | 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. Note: By default,
this uses Spark's default number of parallel tasks (2 for local mode, and in cluster mode the number
is determined by the config property spark.default.parallelism ) to do the grouping.
You can pass an optional numTasks argument to set a different number of tasks. |
join(otherStream, [numTasks]) | 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. |
cogroup(otherStream, [numTasks]) | When called on a DStream of (K, V) and (K, W) pairs, return a new DStream of (K, Seq[V], Seq[W]) tuples. |
transform(func) | 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. |
updateStateByKey(func) | Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values for the key. This can be used to maintain arbitrary state data for each key. |
A few of these transformations are worth discussing in more detail.
UpdateStateByKey Operation
{:.no_toc}
The updateStateByKey
operation allows you to maintain arbitrary state while continuously updating
it with new information. To use this, you will have to do two steps.
- Define the state - The state can be an arbitrary data type.
- Define the state update function - Specify with a function how to update the state using the previous state and the new values from an input stream.
In every batch, Spark will apply the state update function for all existing keys, regardless of whether they have new data in a batch or not. If the update function returns None
then the key-value pair will be eliminated.
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:
{% 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 %}
This is applied on a DStream containing words (say, the pairs
DStream containing (word, 1)
pairs in the earlier example).
{% highlight scala %} val runningCounts = pairs.updateStateByKeyInt {% endhighlight %}
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.
{% highlight java %} Function2<List, Optional, Optional> updateFunction = new Function2<List, Optional, Optional>() { @Override public Optional call(List values, Optional state) { Integer newSum = ... // add the new values with the previous running count to get the new count return Optional.of(newSum); } }; {% endhighlight %}
This is applied on a DStream containing words (say, the pairs
DStream containing (word, 1)
pairs in the quick example).
{% highlight java %} JavaPairDStream<String, Integer> runningCounts = pairs.updateStateByKey(updateFunction); {% endhighlight %}
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
Java code, take a look at the example
[JavaStatefulNetworkWordCount.java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming
/JavaStatefulNetworkWordCount.java).
{% highlight python %} def updateFunction(newValues, runningCount): if runningCount is None: runningCount = 0 return sum(newValues, runningCount) # add the new values with the previous running count to get the new count {% endhighlight %}
This is applied on a DStream containing words (say, the pairs
DStream containing (word, 1)
pairs in the earlier example).
{% highlight python %} runningCounts = pairs.updateStateByKey(updateFunction) {% endhighlight %}
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
Python code, take a look at the example
stateful_network_wordcount.py.
Note that using updateStateByKey
requires the checkpoint directory to be configured, which is
discussed in detail in the checkpointing section.
Transform Operation
{:.no_toc}
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,
one can 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.
{% highlight scala %} val spamInfoRDD = ssc.sparkContext.newAPIHadoopRDD(...) // RDD containing spam information
val cleanedDStream = wordCounts.transform(rdd => { rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning ... }) {% endhighlight %}
{% highlight java %} import org.apache.spark.streaming.api.java.*; // RDD containing spam information final JavaPairRDD<String, Double> spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...);
JavaPairDStream<String, Integer> cleanedDStream = wordCounts.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 %}
{% highlight python %} spamInfoRDD = sc.pickleFile(...) # RDD containing spam information
join data stream with spam information to do data cleaning
cleanedDStream = wordCounts.transform(lambda rdd: rdd.join(spamInfoRDD).filter(...)) {% endhighlight %}
Note that the supplied function gets called in every batch interval. This allows you to do time-varying RDD operations, that is, RDD operations, number of partitions, broadcast variables, etc. can be changed between batches.
Window Operations
{:.no_toc} Spark Streaming also provides windowed computations, which allow you to apply transformations over a sliding window of data. The following figure illustrates this sliding window.
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 the last 3 time units of data, and slides by 2 time units. This shows that any window operation needs to specify two parameters.
- window length - The duration of the window (3 in the figure).
- sliding interval - The interval at which the window 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 by generating word counts over the 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
.
{% highlight scala %} // Reduce last 30 seconds of data, every 10 seconds val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Seconds(30), Seconds(10)) {% endhighlight %}
{% 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) { return i1 + i2; } };
// Reduce last 30 seconds of data, every 10 seconds JavaPairDStream<String, Integer> windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, Durations.seconds(30), Durations.seconds(10)); {% endhighlight %}
{% highlight python %}
Reduce last 30 seconds of data, every 10 seconds
windowedWordCounts = pairs.reduceByKeyAndWindow(lambda x, y: x + y, lambda x, y: x - y, 30, 10) {% endhighlight %}
Some of the common window operations are as follows. All of these operations take the said two parameters - windowLength and slideInterval.
Transformation | Meaning |
---|---|
window(windowLength, slideInterval) | Return a new DStream which is computed based on windowed batches of the source DStream. |
countByWindow(windowLength, slideInterval) | Return a sliding window count of elements in the stream. |
reduceByWindow(func, windowLength, slideInterval) | Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using func. The function should be associative and commutative so that it can be computed correctly in parallel. |
reduceByKeyAndWindow(func, windowLength, slideInterval, [numTasks]) | When called on a DStream of (K, V) pairs, returns a new DStream of (K, V)
pairs where the values for each key are aggregated using the given reduce function func
over batches in a sliding window. Note: By default, this uses Spark's default number of
parallel tasks (2 for local mode, and in cluster mode the number is determined by the config
property spark.default.parallelism ) to do the grouping. You can pass an optional
numTasks argument to set a different number of tasks.
|
reduceByKeyAndWindow(func, invFunc, windowLength, slideInterval, [numTasks]) | A more efficient version of the above reduceByKeyAndWindow() where the reduce
value of each window is calculated incrementally using the reduce values of the previous window.
This is done by reducing the new data that enters the sliding window, and "inverse reducing" the
old data that leaves the window. An example would be that of "adding" and "subtracting" counts
of keys as the window slides. However, it is applicable only to "invertible reduce functions",
that is, those reduce functions which have a corresponding "inverse reduce" function (taken as
parameter invFunc). Like in reduceByKeyAndWindow , the number of reduce tasks
is configurable through an optional argument. Note that checkpointing must be
enabled for using this operation.
|
countByValueAndWindow(windowLength, slideInterval, [numTasks]) | When called on a DStream of (K, V) pairs, returns a new DStream of (K, Long) pairs where the
value of each key is its frequency within a sliding window. Like in
reduceByKeyAndWindow , the number of reduce tasks is configurable through an
optional argument.
|
Join Operations
{:.no_toc} Finally, its worth highlighting how easily you can perform different kinds of joins in Spark Streaming.
Stream-stream joins
{:.no_toc} Streams can be very easily joined with other streams.
Stream-dataset joins
{:.no_toc}
This has already been shown earlier while explain DStream.transform
operation. Here is yet another example of joining a windowed stream with a dataset.
In fact, you can also dynamically change the dataset you want to join against. The function provided to transform
is evaluated every batch interval and therefore will use the current dataset that dataset
reference points to.
The complete list of DStream transformations is available in the API documentation. For the Scala API, see DStream and PairDStreamFunctions. For the Java API, see JavaDStream and JavaPairDStream. For the Python API, see DStream.
Output Operations on DStreams
Output operations allow DStream's data to be pushed out to external systems like a database or a file systems. Since the output operations actually allow the transformed data to be consumed by external systems, they trigger the actual execution of all the DStream transformations (similar to actions for RDDs). Currently, the following output operations are defined:
Output Operation | Meaning |
---|---|
print() | Prints the first ten elements of every batch of data in a DStream on the driver node running
the streaming application. This is useful for development and debugging.
Python API This is called pprint() in the Python API. |
saveAsTextFiles(prefix, [suffix]) | Save this DStream's contents as text files. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]". |
saveAsObjectFiles(prefix, [suffix]) | Save this DStream's contents as SequenceFiles of serialized Java objects. The file
name at each batch interval is generated based on prefix and
suffix: "prefix-TIME_IN_MS[.suffix]".
Python API This is not available in the Python API. |
saveAsHadoopFiles(prefix, [suffix]) | Save this DStream's contents as Hadoop files. The file name at each batch interval is
generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
Python API This is not available in the Python API. |
foreachRDD(func) | The most generic output operator that applies a function, func, to each RDD generated from the stream. This function should push the data in each RDD to an external system, such as saving the RDD to files, or writing it over the network to a database. Note that the function func is executed in the driver process running the streaming application, and will usually have RDD actions in it that will force the computation of the streaming RDDs. |
Design Patterns for using foreachRDD
{:.no_toc}
dstream.foreachRDD
is a powerful primitive that allows data to be sent out to external systems.
However, it is important to understand how to use this primitive correctly and efficiently.
Some of the common mistakes to avoid are as follows.
Often writing data to external system requires creating a connection object (e.g. TCP connection to a remote server) and using it to send data to a remote system. For this purpose, a developer may inadvertently try creating a connection object at the Spark driver, and then try to use it in a Spark worker to save records in the RDDs. For example (in Scala),
dstream.foreachRDD(sendRecord) {% endhighlight %}
This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker.
However, this can lead to another common mistake - creating a new connection for every record. For example,
dstream.foreachRDD(lambda rdd: rdd.foreach(sendRecord)) {% endhighlight %}
Typically, creating a connection object has time and resource overheads. Therefore, creating and
destroying a connection object for each record can incur unnecessarily high overheads and can
significantly reduce the overall throughput of the system. A better solution is to use
rdd.foreachPartition
- create a single connection object and send all the records in a RDD
partition using that connection.
dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) {% endhighlight %}
This amortizes the connection creation overheads over many records.
Finally, this can be further optimized by reusing connection objects across multiple RDDs/batches. One can maintain a static pool of connection objects than can be reused as RDDs of multiple batches are pushed to the external system, thus further reducing the overheads.
dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) {% endhighlight %}
Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems.
Other points to remember:
{:.no_toc}
-
DStreams are executed lazily by the output operations, just like RDDs are lazily executed by RDD actions. Specifically, RDD actions inside the DStream output operations force the processing of the received data. Hence, if your application does not have any output operation, or has output operations like
dstream.foreachRDD()
without any RDD action inside them, then nothing will get executed. The system will simply receive the data and discard it. -
By default, output operations are executed one-at-a-time. And they are executed in the order they are defined in the application.
Accumulators and Broadcast Variables
Accumulators and Broadcast variables cannot be recovered from checkpoint in Spark Streaming. If you enable checkpointing and use Accumulators or Broadcast variables as well, you'll have to create lazily instantiated singleton instances for Accumulators and Broadcast variables so that they can be re-instantiated after the driver restarts on failure. This is shown in the following example.
object WordBlacklist {
@volatile private var instance: Broadcast[Seq[String]] = null
def getInstance(sc: SparkContext): Broadcast[Seq[String]] = { if (instance == null) { synchronized { if (instance == null) { val wordBlacklist = Seq("a", "b", "c") instance = sc.broadcast(wordBlacklist) } } } instance } }
object DroppedWordsCounter {
@volatile private var instance: LongAccumulator = null
def getInstance(sc: SparkContext): LongAccumulator = { if (instance == null) { synchronized { if (instance == null) { instance = sc.longAccumulator("WordsInBlacklistCounter") } } } instance } }
wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => // Get or register the blacklist Broadcast val blacklist = WordBlacklist.getInstance(rdd.sparkContext) // Get or register the droppedWordsCounter Accumulator val droppedWordsCounter = DroppedWordsCounter.getInstance(rdd.sparkContext) // Use blacklist to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => if (blacklist.value.contains(word)) { droppedWordsCounter.add(count) false } else { true } }.collect().mkString("[", ", ", "]") val output = "Counts at time " + time + " " + counts })
{% endhighlight %}
See the full source code.
class JavaWordBlacklist {
private static volatile Broadcast<List> instance = null;
public static Broadcast<List> getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (JavaWordBlacklist.class) { if (instance == null) { List wordBlacklist = Arrays.asList("a", "b", "c"); instance = jsc.broadcast(wordBlacklist); } } } return instance; } }
class JavaDroppedWordsCounter {
private static volatile LongAccumulator instance = null;
public static LongAccumulator getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (JavaDroppedWordsCounter.class) { if (instance == null) { instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); } } } return instance; } }
wordCounts.foreachRDD(new Function2<JavaPairRDD<String, Integer>, Time, Void>() { @Override public Void call(JavaPairRDD<String, Integer> rdd, Time time) throws IOException { // Get or register the blacklist Broadcast final Broadcast<List> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator final LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); // Use blacklist to drop words and use droppedWordsCounter to count them String counts = rdd.filter(new Function<Tuple2<String, Integer>, Boolean>() { @Override public Boolean call(Tuple2<String, Integer> wordCount) throws Exception { if (blacklist.value().contains(wordCount._1())) { droppedWordsCounter.add(wordCount._2()); return false; } else { return true; } } }).collect().toString(); String output = "Counts at time " + time + " " + counts; } }
{% endhighlight %}
See the full source code.
def getWordBlacklist(sparkContext): if ('wordBlacklist' not in globals()): globals()['wordBlacklist'] = sparkContext.broadcast(["a", "b", "c"]) return globals()['wordBlacklist']
def getDroppedWordsCounter(sparkContext): if ('droppedWordsCounter' not in globals()): globals()['droppedWordsCounter'] = sparkContext.accumulator(0) return globals()['droppedWordsCounter']
def echo(time, rdd): # Get or register the blacklist Broadcast blacklist = getWordBlacklist(rdd.context) # Get or register the droppedWordsCounter Accumulator droppedWordsCounter = getDroppedWordsCounter(rdd.context)
# Use blacklist to drop words and use droppedWordsCounter to count them
def filterFunc(wordCount):
if wordCount[0] in blacklist.value:
droppedWordsCounter.add(wordCount[1])
False
else:
True
counts = "Counts at time %s %s" % (time, rdd.filter(filterFunc).collect())
wordCounts.foreachRDD(echo)
{% endhighlight %}
See the full source code.