Skip to content
Snippets Groups Projects
streaming-programming-guide.md 120.65 KiB
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, Twitter, ZeroMQ, 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.

Spark Streaming

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

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.

First, we import the names of the Spark Streaming classes and some implicit conversions from StreamingContext into our environment in order to add useful methods to other classes we need (like DStream). StreamingContext 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 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 the lines by space into words.

{% highlight java %} // Split each line into words JavaDStream words = lines.flatMap( new FlatMapFunction<String, String>() { @Override public Iterable call(String x) { return Arrays.asList(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 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<String, Integer>(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.

First, we import StreamingContext, which is the main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second.

{% 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

{% highlight bash %} $ ./bin/run-example streaming.NetworkWordCount localhost 9999 {% endhighlight %}
{% highlight bash %} $ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %}
{% highlight bash %} $ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 {% endhighlight %}

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_{{site.SCALA_BINARY_VERSION}}
Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
Kinesis
spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]
Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}

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, 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.

  1. Define the input sources by creating input DStreams.
  2. Define the streaming computations by applying transformation and output operations to DStreams.
  3. Start receiving data and processing it using streamingContext.start().
  4. Wait for the processing to be stopped (manually or due to any error) using streamingContext.awaitTermination().
  5. 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() called stopSparkContext 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.

Spark Streaming

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.

Spark Streaming

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, socket connections, and Akka actors.
  • Advanced sources: Sources like Kafka, Flume, Kinesis, Twitter, 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 a 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 and Akka actors 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, InputFormatClass
    streamingContext.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, only textFileStream is available.

  • Streams based on Custom Actors: DStreams can be created with data streams received through Akka actors by using streamingContext.actorStream(actorProps, actor-name). See the Custom Receiver Guide for more details.

    Python API Since actors are available only in the Java and Scala libraries, actorStream is not available in the Python API.

  • 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.