Skip to content
Snippets Groups Projects
structured-streaming-programming-guide.md 68.68 KiB
layout: global
displayTitle: Structured Streaming Programming Guide [Alpha]
title: Structured Streaming Programming Guide
  • This will become a table of contents (this text will be scraped). {:toc}

Overview

Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. You can express your streaming computation the same way you would express a batch computation on static data.The Spark SQL engine will take care of running it incrementally and continuously and updating the final result as streaming data continues to arrive. You can use the Dataset/DataFrame API in Scala, Java or Python to express streaming aggregations, event-time windows, stream-to-batch joins, etc. The computation is executed on the same optimized Spark SQL engine. Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees through checkpointing and Write Ahead Logs. In short, Structured Streaming provides fast, scalable, fault-tolerant, end-to-end exactly-once stream processing without the user having to reason about streaming.

Structured Streaming is still ALPHA in Spark 2.1 and the APIs are still experimental. In this guide, we are going to walk you through the programming model and the APIs. First, let's start with a simple example - a streaming word count.

Quick Example

Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in Scala/Java/Python. And if you download Spark, you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark.

{% highlight scala %} import org.apache.spark.sql.functions._ import org.apache.spark.sql.SparkSession

val spark = SparkSession .builder .appName("StructuredNetworkWordCount") .getOrCreate()

import spark.implicits._ {% endhighlight %}

{% highlight java %} import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.sql.*; import org.apache.spark.sql.streaming.StreamingQuery;

import java.util.Arrays; import java.util.Iterator;

SparkSession spark = SparkSession .builder() .appName("JavaStructuredNetworkWordCount") .getOrCreate(); {% endhighlight %}

{% highlight python %} from pyspark.sql import SparkSession from pyspark.sql.functions import explode from pyspark.sql.functions import split

spark = SparkSession
.builder
.appName("StructuredNetworkWordCount")
.getOrCreate() {% endhighlight %}

Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts.

{% highlight scala %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 val lines = spark.readStream .format("socket") .option("host", "localhost") .option("port", 9999) .load()

// Split the lines into words val words = lines.as[String].flatMap(_.split(" "))

// Generate running word count val wordCounts = words.groupBy("value").count() {% endhighlight %}

This lines DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using .as[String], so that we can apply the flatMap operation to split each line into multiple words. The resultant words Dataset contains all the words. Finally, we have defined the wordCounts DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.

{% highlight java %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 Dataset lines = spark .readStream() .format("socket") .option("host", "localhost") .option("port", 9999) .load();

// Split the lines into words Dataset words = lines .as(Encoders.STRING()) .flatMap( new FlatMapFunction<String, String>() { @Override public Iterator call(String x) { return Arrays.asList(x.split(" ")).iterator(); } }, Encoders.STRING());

// Generate running word count Dataset wordCounts = words.groupBy("value").count(); {% endhighlight %}

This lines DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using .as(Encoders.STRING()), so that we can apply the flatMap operation to split each line into multiple words. The resultant words Dataset contains all the words. Finally, we have defined the wordCounts DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.

{% highlight python %}

Create DataFrame representing the stream of input lines from connection to localhost:9999

lines = spark
.readStream
.format("socket")
.option("host", "localhost")
.option("port", 9999)
.load()

Split the lines into words

words = lines.select( explode( split(lines.value, " ") ).alias("word") )

Generate running word count

wordCounts = words.groupBy("word").count() {% endhighlight %}

This lines DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function alias to name the new column as "word". Finally, we have defined the wordCounts DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.

We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by outputMode("complete")) to the console every time they are updated. And then start the streaming computation using start().

{% highlight scala %} // Start running the query that prints the running counts to the console val query = wordCounts.writeStream .outputMode("complete") .format("console") .start()

query.awaitTermination() {% endhighlight %}

{% highlight java %} // Start running the query that prints the running counts to the console StreamingQuery query = wordCounts.writeStream() .outputMode("complete") .format("console") .start();

query.awaitTermination(); {% endhighlight %}

{% highlight python %}

Start running the query that prints the running counts to the console

query = wordCounts
.writeStream
.outputMode("complete")
.format("console")
.start()

query.awaitTermination() {% endhighlight %}

After this code is executed, the streaming computation will have started in the background. The query object is a handle to that active streaming query, and we have decided to wait for the termination of the query using query.awaitTermination() to prevent the process from exiting while the query is active.

To actually execute this example code, you can either compile the code in your own Spark application, or simply run the example once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using

$ nc -lk 9999

Then, in a different terminal, you can start the example by using

{% highlight bash %} $ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 {% endhighlight %}
{% highlight bash %} $ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 {% endhighlight %}
{% highlight bash %} $ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_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 apache spark apache hadoop

... {% endhighlight %}

{% highlight bash %} # TERMINAL 2: RUNNING StructuredNetworkWordCount

$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999


Batch: 0

+------+-----+ | value|count| +------+-----+ |apache| 1| | spark| 1| +------+-----+


Batch: 1

+------+-----+ | value|count| +------+-----+ |apache| 2| | spark| 1| |hadoop| 1| +------+-----+ ... {% endhighlight %}

{% highlight bash %} # TERMINAL 2: RUNNING JavaStructuredNetworkWordCount

$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999


Batch: 0

+------+-----+ | value|count| +------+-----+ |apache| 1| | spark| 1| +------+-----+


Batch: 1

+------+-----+ | value|count| +------+-----+ |apache| 2| | spark| 1| |hadoop| 1| +------+-----+ ... {% endhighlight %}

{% highlight bash %} # TERMINAL 2: RUNNING structured_network_wordcount.py

$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999


Batch: 0

+------+-----+ | value|count| +------+-----+ |apache| 1| | spark| 1| +------+-----+


Batch: 1

+------+-----+ | value|count| +------+-----+ |apache| 2| | spark| 1| |hadoop| 1| +------+-----+ ... {% endhighlight %}

Programming Model

The key idea in Structured Streaming is to treat a live data stream as a table that is being continuously appended. This leads to a new stream processing model that is very similar to a batch processing model. You will express your streaming computation as standard batch-like query as on a static table, and Spark runs it as an incremental query on the unbounded input table. Let’s understand this model in more detail.

Basic Concepts

Consider the input data stream as the "Input Table". Every data item that is arriving on the stream is like a new row being appended to the Input Table.

Stream as a Table

A query on the input will generate the "Result Table". Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink.

Model

The "Output" is defined as what gets written out to the external storage. The output can be defined in different modes

  • Complete Mode - The entire updated Result Table will be written to the external storage. It is up to the storage connector to decide how to handle writing of the entire table.

  • Append Mode - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on the queries where existing rows in the Result Table are not expected to change.

  • Update Mode - Only the rows that were updated in the Result Table since the last trigger will be written to the external storage (available since Spark 2.1.1). Note that this is different from the Complete Mode in that this mode only outputs the rows that have changed since the last trigger.

Note that each mode is applicable on certain types of queries. This is discussed in detail later.

To illustrate the use of this model, let’s understand the model in context of the Quick Example above. The first lines DataFrame is the input table, and the final wordCounts DataFrame is the result table. Note that the query on streaming lines DataFrame to generate wordCounts is exactly the same as it would be a static DataFrame. However, when this query is started, Spark will continuously check for new data from the socket connection. If there is new data, Spark will run an "incremental" query that combines the previous running counts with the new data to compute updated counts, as shown below.

Model

This model is significantly different from many other stream processing engines. Many streaming systems require the user to maintain running aggregations themselves, thus having to reason about fault-tolerance, and data consistency (at-least-once, or at-most-once, or exactly-once). In this model, Spark is responsible for updating the Result Table when there is new data, thus relieving the users from reasoning about it. As an example, let’s see how this model handles event-time based processing and late arriving data.

Handling Event-time and Late Data

Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of events every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on a data stream, making the life of the user much easier.

Furthermore, this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating old aggregates when there is late data, as well as cleaning up old aggregates to limit the size of intermediate state data. Since Spark 2.1, we have support for watermarking which allows the user to specify the threshold of late data, and allows the engine to accordingly clean up old state. These are explained later in more details in the Window Operations section.

Fault Tolerance Semantics

Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotent sinks, Structured Streaming can ensure end-to-end exactly-once semantics under any failure.

API using Datasets and DataFrames

Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point SparkSession (Scala/Java/Python docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the DataFrame/Dataset Programming Guide.

Creating streaming DataFrames and streaming Datasets

Streaming DataFrames can be created through the DataStreamReader interface (Scala/Java/Python docs) returned by SparkSession.readStream(). Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc.

Input Sources

In Spark 2.0, there are a few built-in sources.

  • File source - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations.

  • Kafka source - Poll data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the Kafka Integration Guide for more details.

  • Socket source (for testing) - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees.

Some sources are not fault-tolerant because they do not guarantee that data can be replayed using checkpointed offsets after a failure. See the earlier section on fault-tolerance semantics. Here are the details of all the sources in Spark.

Source Options Fault-tolerant Notes
File source path: path to the input directory, and common to all file formats.

For file-format-specific options, see the related methods in DataStreamReader (Scala/Java/Python). E.g. for "parquet" format options see DataStreamReader.parquet()
Yes Supports glob paths, but does not support multiple comma-separated paths/globs.
Socket Source host: host to connect to, must be specified
port: port to connect to, must be specified
No
Kafka Source See the Kafka Integration Guide. Yes

Here are some examples.

{% highlight scala %} val spark: SparkSession = ...

// Read text from socket val socketDF = spark .readStream .format("socket") .option("host", "localhost") .option("port", 9999) .load()

socketDF.isStreaming // Returns True for DataFrames that have streaming sources

socketDF.printSchema

// Read all the csv files written atomically in a directory val userSchema = new StructType().add("name", "string").add("age", "integer") val csvDF = spark .readStream .option("sep", ";") .schema(userSchema) // Specify schema of the csv files .csv("/path/to/directory") // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}

{% highlight java %} SparkSession spark = ...

// Read text from socket Dataset[Row] socketDF = spark .readStream() .format("socket") .option("host", "localhost") .option("port", 9999) .load();

socketDF.isStreaming(); // Returns True for DataFrames that have streaming sources

socketDF.printSchema();

// Read all the csv files written atomically in a directory StructType userSchema = new StructType().add("name", "string").add("age", "integer"); Dataset[Row] csvDF = spark .readStream() .option("sep", ";") .schema(userSchema) // Specify schema of the csv files .csv("/path/to/directory"); // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}

{% highlight python %} spark = SparkSession. ...

Read text from socket

socketDF = spark
.readStream()
.format("socket")
.option("host", "localhost")
.option("port", 9999)
.load()

socketDF.isStreaming() # Returns True for DataFrames that have streaming sources

socketDF.printSchema()

Read all the csv files written atomically in a directory

userSchema = StructType().add("name", "string").add("age", "integer") csvDF = spark
.readStream()
.option("sep", ";")
.schema(userSchema)
.csv("/path/to/directory") # Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}

These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like map, flatMap, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document.

Schema inference and partition of streaming DataFrames/Datasets

By default, Structured Streaming from file based sources requires you to specify the schema, rather than rely on Spark to infer it automatically. This restriction ensures a consistent schema will be used for the streaming query, even in the case of failures. For ad-hoc use cases, you can reenable schema inference by setting spark.sql.streaming.schemaInference to true.

Partition discovery does occur when subdirectories that are named /key=value/ are present and listing will automatically recurse into these directories. If these columns appear in the user provided schema, they will be filled in by Spark based on the path of the file being read. The directories that make up the partitioning scheme must be present when the query starts and must remain static. For example, it is okay to add /data/year=2016/ when /data/year=2015/ was present, but it is invalid to change the partitioning column (i.e. by creating the directory /data/date=2016-04-17/).

Operations on streaming DataFrames/Datasets

You can apply all kinds of operations on streaming DataFrames/Datasets – ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use.

Basic Operations - Selection, Projection, Aggregation

Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are discussed later in this section.

{% highlight scala %} case class DeviceData(device: String, deviceType: String, signal: Double, time: DateTime)

val df: DataFrame = ... // streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: string } val ds: Dataset[DeviceData] = df.as[DeviceData] // streaming Dataset with IOT device data

// Select the devices which have signal more than 10 df.select("device").where("signal > 10") // using untyped APIs
ds.filter(.signal > 10).map(.device) // using typed APIs

// Running count of the number of updates for each device type df.groupBy("deviceType").count() // using untyped API

// Running average signal for each device type import org.apache.spark.sql.expressions.scalalang.typed ds.groupByKey(.deviceType).agg(typed.avg(.signal)) // using typed API {% endhighlight %}

{% highlight java %} import org.apache.spark.api.java.function.; import org.apache.spark.sql.; import org.apache.spark.sql.expressions.javalang.typed; import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;

public class DeviceData { private String device; private String deviceType; private Double signal; private java.sql.Date time; ... // Getter and setter methods for each field }

Dataset df = ...; // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } Dataset ds = df.as(ExpressionEncoder.javaBean(DeviceData.class)); // streaming Dataset with IOT device data

// Select the devices which have signal more than 10 df.select("device").where("signal > 10"); // using untyped APIs ds.filter(new FilterFunction() { // using typed APIs @Override public boolean call(DeviceData value) throws Exception { return value.getSignal() > 10; } }).map(new MapFunction<DeviceData, String>() { @Override public String call(DeviceData value) throws Exception { return value.getDevice(); } }, Encoders.STRING());

// Running count of the number of updates for each device type df.groupBy("deviceType").count(); // using untyped API

// Running average signal for each device type ds.groupByKey(new MapFunction<DeviceData, String>() { // using typed API @Override public String call(DeviceData value) throws Exception { return value.getDeviceType(); } }, Encoders.STRING()).agg(typed.avg(new MapFunction<DeviceData, Double>() { @Override public Double call(DeviceData value) throws Exception { return value.getSignal(); } })); {% endhighlight %}

{% highlight python %} df = ... # streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: DateType }

Select the devices which have signal more than 10

df.select("device").where("signal > 10")

Running count of the number of updates for each device type

df.groupBy("deviceType").count() {% endhighlight %}

Window Operations on Event Time

Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration.

Imagine our quick example is modified and the stream now contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time).