Structured Streaming Programming Guide - Spark 3.4.0 Documentation
Structured Streaming Programming Guide - Spark 3.4.0 Documentation
3.4.0
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,
Python or R 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.
Internally, by default, Structured Streaming queries are processed using a micro-batch processing engine, which processes data streams as a series
of small batch jobs thereby achieving end-to-end latencies as low as 100 milliseconds and exactly-once fault-tolerance guarantees. However,
since Spark 2.3, we have introduced a new low-latency processing mode called Continuous Processing, which can achieve end-to-end latencies
as low as 1 millisecond with at-least-once guarantees. Without changing the Dataset/DataFrame operations in your queries, you will be able to
choose the mode based on your application requirements.
In this guide, we are going to walk you through the programming model and the APIs. We are going to explain the concepts mostly using the
default micro-batch processing model, and then later discuss Continuous Processing model. First, let’s start with a simple example of a Structured
Streaming query - 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/R. 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.
import org.apache.spark.sql.functions._
import org.apache.spark.sql.SparkSession
import spark.implicits._
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.
// 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()
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.
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().
// Start running the query that prints the running counts to the console
val query = wordCounts.writeStream
.outputMode("complete")
.format("console")
.start()
query.awaitTermination()
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 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, 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.
-------------------------------------------
Batch: 1
-------------------------------------------
+------+-----+
| value|count|
+------+-----+
|apache| 2|
| spark| 1|
|hadoop| 1|
... +------+-----+
...
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.
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.
The “Output” is defined as what gets written out to the external storage. The output can be defined in a different mode:
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. If the query doesn’t contain aggregations, it will be equivalent to Append mode.
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.
Note that Structured Streaming does not materialize the entire table. It reads the latest available data from the streaming data source,
processes it incrementally to update the result, and then discards the source data. It only keeps around the minimal intermediate state data as
required to update the result (e.g. intermediate counts in the earlier example).
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.
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 detail in the Window Operations section.
Input Sources
There are a few built-in sources.
File source - Reads files written in a directory as a stream of data. Files will be processed in the order of file modification time. If latestFirst
is set, order will be reversed. Supported file formats are text, CSV, JSON, ORC, 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 - Reads 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.
Rate source (for testing) - Generates data at the specified number of rows per second, each output row contains a timestamp and value.
Where timestamp is a Timestamp type containing the time of message dispatch, and value is of Long type containing the message count,
starting from 0 as the first row. This source is intended for testing and benchmarking.
Rate Per Micro-Batch source (for testing) - Generates data at the specified number of rows per micro-batch, each output row contains a
timestamp and value. Where timestamp is a Timestamp type containing the time of message dispatch, and value is of Long type containing
the message count, starting from 0 as the first row. Unlike rate data source, this data source provides a consistent set of input rows per
micro-batch regardless of query execution (configuration of trigger, query being lagging, etc.), say, batch 0 will produce 0~999 and batch 1
will produce 1000~1999, and so on. Same applies to the generated time. This source is intended for testing and benchmarking.
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.
File source path: path to the input directory, and common to all file formats. Yes Supports glob
maxFilesPerTrigger: maximum number of new files to be considered in every trigger paths, but does
(default: no max) not support
latestFirst: whether to process the latest new files first, useful when there is a large multiple comma-
backlog of files (default: false) separated
fileNameOnly: whether to check new files based on only the filename instead of on the paths/globs.
full path (default: false). With this set to `true`, the following files would be considered as
the same file, because their filenames, "dataset.txt", are the same:
"file:///dataset.txt"
"s3://a/dataset.txt"
"s3n://a/b/dataset.txt"
"s3a://a/b/c/dataset.txt"
maxFileAge: Maximum age of a file that can be found in this directory, before it is
ignored. For the first batch all files will be considered valid. If latestFirst is set to `true`
and maxFilesPerTrigger is set, then this parameter will be ignored, because old files that
are valid, and should be processed, may be ignored. The max age is specified with respect
to the timestamp of the latest file, and not the timestamp of the current system.(default: 1
week)
cleanSource: option to clean up completed files after processing.
Available options are "archive", "delete", "off". If the option is not provided, the default
value is "off".
When "archive" is provided, additional option sourceArchiveDir must be provided as
well. The value of "sourceArchiveDir" must not match with source pattern in depth (the
number of directories from the root directory), where the depth is minimum of depth on
both paths. This will ensure archived files are never included as new source files.
For example, suppose you provide '/hello?/spark/*' as source pattern,
'/hello1/spark/archive/dir' cannot be used as the value of "sourceArchiveDir", as
'/hello?/spark/*' and '/hello1/spark/archive' will be matched. '/hello1/spark' cannot be also
used as the value of "sourceArchiveDir", as '/hello?/spark' and '/hello1/spark' will be
matched. '/archived/here' would be OK as it doesn't match.
Spark will move source files respecting their own path. For example, if the path of source
file is /a/b/dataset.txt and the path of archive directory is /archived/here, file will be
moved to /archived/here/a/b/dataset.txt.
NOTE: Both archiving (via moving) or deleting completed files will introduce overhead
(slow down, even if it's happening in separate thread) in each micro-batch, so you need to
understand the cost for each operation in your file system before enabling this option. On
the other hand, enabling this option will reduce the cost to list source files which can be
an expensive operation.
Number of threads used in completed file cleaner can be configured with
spark.sql.streaming.fileSource.cleaner.numThreads (default: 1).
NOTE 2: The source path should not be used from multiple sources or queries when
enabling this option. Similarly, you must ensure the source path doesn't match to any files
in output directory of file stream sink.
NOTE 3: Both delete and move actions are best effort. Failing to delete or move files will
not fail the streaming query. Spark may not clean up some source files in some
circumstances - e.g. the application doesn't shut down gracefully, too many files are
queued to clean up.
In addition, there are session configurations that affect certain file-formats. See the SQL
Programming Guide for more details. E.g., for "parquet", see Parquet configuration
section.
Rate rowsPerSecond (e.g. 100, default: 1): How many rows should be generated per second. Yes
Source
rampUpTime (e.g. 5s, default: 0s): How long to ramp up before the generating speed
becomes rowsPerSecond. Using finer granularities than seconds will be truncated to
integer seconds.
numPartitions (e.g. 10, default: Spark's default parallelism): The partition number for the
generated rows.
The source will try its best to reach rowsPerSecond, but the query may be resource
constrained, and numPartitions can be tweaked to help reach the desired speed.
Rate Per rowsPerBatch (e.g. 100): How many rows should be generated per micro-batch. Yes
Micro-
Batch numPartitions (e.g. 10, default: Spark's default parallelism): The partition number for the
Source generated rows.
(format:
rate- startTimestamp (e.g. 1000, default: 0): starting value of generated time.
micro-
batch) advanceMillisPerBatch (e.g. 1000, default: 1000): the amount of time being advanced in
generated time on each micro-batch.
socketDF.printSchema()
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.
Since Spark 3.1, you can also create streaming DataFrames from tables with DataStreamReader.table(). See Streaming Table APIs for more
details.
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/).
df = ... # streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time
You can also register a streaming DataFrame/Dataset as a temporary view and then apply SQL commands on it.
df.createOrReplaceTempView("updates")
spark.sql("select count(*) from updates") // returns another streaming DF
Note, you can identify whether a DataFrame/Dataset has streaming data or not by using df.isStreaming.
df.isStreaming
You may want to check the query plan of the query, as Spark could inject stateful operations during interpret of SQL statement against streaming
dataset. Once stateful operations are injected in the query plan, you may need to check your query with considerations in stateful operations. (e.g.
output mode, watermark, state store size maintenance, etc.)
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).
Since this windowing is similar to grouping, in code, you can use groupBy() and window() operations to express windowed aggregations. You
can see the full code for the below examples in Scala/Java/Python.
import spark.implicits._
val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String }
// Group the data by window and word and compute the count of each group
val windowedCounts = words.groupBy(
window($"timestamp", "10 minutes", "5 minutes"),
$"word"
).count()
However, to run this query for days, it’s necessary for the system to bound the amount of intermediate in-memory state it accumulates. This
means the system needs to know when an old aggregate can be dropped from the in-memory state because the application is not going to
receive late data for that aggregate any more. To enable this, in Spark 2.1, we have introduced watermarking, which lets the engine automatically
track the current event time in the data and attempt to clean up old state accordingly. You can define the watermark of a query by specifying the
event time column and the threshold on how late the data is expected to be in terms of event time. For a specific window ending at time T, the
engine will maintain state and allow late data to update the state until (max event time seen by the engine - late threshold > T). In
other words, late data within the threshold will be aggregated, but data later than the threshold will start getting dropped (see later in the section
for the exact guarantees). Let’s understand this with an example. We can easily define watermarking on the previous example using
withWatermark() as shown below.
# Group the data by window and word and compute the count of each group
windowedCounts = words \
.withWatermark("timestamp", "10 minutes") \
.groupBy(
window(words.timestamp, "10 minutes", "5 minutes"),
words.word) \
.count()
In this example, we are defining the watermark of the query on the value of the column “timestamp”, and also defining “10 minutes” as the
threshold of how late is the data allowed to be. If this query is run in Update output mode (discussed later in Output Modes section), the engine
will keep updating counts of a window in the Result Table until the window is older than the watermark, which lags behind the current event time
in column “timestamp” by 10 minutes. Here is an illustration.
As shown in the illustration, the maximum event time tracked by the engine is the blue dashed line, and the watermark set as (max event time -
'10 mins') at the beginning of every trigger is the red line. For example, when the engine observes the data (12:14, dog), it sets the
watermark for the next trigger as 12:04. This watermark lets the engine maintain intermediate state for additional 10 minutes to allow late data to
be counted. For example, the data (12:09, cat) is out of order and late, and it falls in windows 12:00 - 12:10 and 12:05 - 12:15. Since, it is
still ahead of the watermark 12:04 in the trigger, the engine still maintains the intermediate counts as state and correctly updates the counts of
the related windows. However, when the watermark is updated to 12:11, the intermediate state for window (12:00 - 12:10) is cleared, and all
subsequent data (e.g. (12:04, donkey)) is considered “too late” and therefore ignored. Note that after every trigger, the updated counts (i.e.
purple rows) are written to sink as the trigger output, as dictated by the Update mode.
Some sinks (e.g. files) may not supported fine-grained updates that Update Mode requires. To work with them, we have also support Append
Mode, where only the final counts are written to sink. This is illustrated below.
Note that using withWatermark on a non-streaming Dataset is no-op. As the watermark should not affect any batch query in any way, we will
ignore it directly.
Similar to the Update Mode earlier, the engine maintains intermediate counts for each window. However, the partial counts are not updated to
the Result Table and not written to sink. The engine waits for “10 mins” for late date to be counted, then drops intermediate state of a window <
watermark, and appends the final counts to the Result Table/sink. For example, the final counts of window 12:00 - 12:10 is appended to the
Result Table only after the watermark is updated to 12:11.
Tumbling windows are a series of fixed-sized, non-overlapping and contiguous time intervals. An input can only be bound to a single window.
Sliding windows are similar to the tumbling windows from the point of being “fixed-sized”, but windows can overlap if the duration of slide is
smaller than the duration of window, and in this case an input can be bound to the multiple windows.
Tumbling and sliding window use window function, which has been described on above examples.
Session windows have different characteristic compared to the previous two types. Session window has a dynamic size of the window length,
depending on the inputs. A session window starts with an input, and expands itself if following input has been received within gap duration. For
static gap duration, a session window closes when there’s no input received within gap duration after receiving the latest input.
Session window uses session_window function. The usage of the function is similar to the window function.
# Group the data by session window and userId, and compute the count of each group
sessionizedCounts = events \
.withWatermark("timestamp", "10 minutes") \
.groupBy(
session_window(events.timestamp, "5 minutes"),
events.userId) \
.count()
Instead of static value, we can also provide an expression to specify gap duration dynamically based on the input row. Note that the rows with
negative or zero gap duration will be filtered out from the aggregation.
With dynamic gap duration, the closing of a session window does not depend on the latest input anymore. A session window’s range is the union
of all events’ ranges which are determined by event start time and evaluated gap duration during the query execution.
session_window = session_window(events.timestamp, \
F.when(events.userId == "user1", "5 seconds") \
.when(events.userId == "user2", "20 seconds").otherwise("5 minutes"))
# Group the data by session window and userId, and compute the count of each group
sessionizedCounts = events \
.withWatermark("timestamp", "10 minutes") \
.groupBy(
session_window,
events.userId) \
.count()
Note that there are some restrictions when you use session window in streaming query, like below:
For batch query, global window (only having session_window in grouping key) is supported.
By default, Spark does not perform partial aggregation for session window aggregation, since it requires additional sort in local partitions before
grouping. It works better for the case there are only few number of input rows in same group key for each local partition, but for the case there
are numerous input rows having same group key in local partition, doing partial aggregation can still increase the performance significantly
despite additional sort.
window_time function will produce a timestamp which represents the time for time window. User can pass the result to the parameter of window
function (or anywhere requiring timestamp) to perform operation(s) with time window which requires timestamp.
import spark.implicits._
val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String }
// Group the data by window and word and compute the count of each group
val windowedCounts = words.groupBy(
window($"timestamp", "10 minutes", "5 minutes"),
$"word"
).count()
// Group the windowed data by another window and word and compute the count of each group
val anotherWindowedCounts = windowedCounts.groupBy(
window(window_time($"window"), "1 hour"),
$"word"
).count()
window function does not only take timestamp column, but also take the time window column. This is specifically useful for cases where users
want to apply chained time window aggregations.
import spark.implicits._
val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String }
// Group the data by window and word and compute the count of each group
val windowedCounts = words.groupBy(
window($"timestamp", "10 minutes", "5 minutes"),
$"word"
).count()
// Group the windowed data by another window and word and compute the count of each group
val anotherWindowedCounts = windowedCounts.groupBy(
window($"window", "1 hour"),
$"word"
).count()
Output mode must be Append or Update. Complete mode requires all aggregate data to be preserved, and hence cannot use
watermarking to drop intermediate state. See the Output Modes section for detailed explanation of the semantics of each output mode.
The aggregation must have either the event-time column, or a window on the event-time column.
withWatermark must be called on the same column as the timestamp column used in the aggregate. For example,
df.withWatermark("time", "1 min").groupBy("time2").count() is invalid in Append output mode, as watermark is defined on a
different column from the aggregation column.
withWatermark must be called before the aggregation for the watermark details to be used. For example,
df.groupBy("time").count().withWatermark("time", "1 min") is invalid in Append output mode.
However, the guarantee is strict only in one direction. Data delayed by more than 2 hours is not guaranteed to be dropped; it may or may not
get aggregated. More delayed is the data, less likely is the engine going to process it.
Join Operations
Structured Streaming supports joining a streaming Dataset/DataFrame with a static Dataset/DataFrame as well as another streaming
Dataset/DataFrame. The result of the streaming join is generated incrementally, similar to the results of streaming aggregations in the previous
section. In this section we will explore what type of joins (i.e. inner, outer, semi, etc.) are supported in the above cases. Note that in all the
supported join types, the result of the join with a streaming Dataset/DataFrame will be the exactly the same as if it was with a static
Dataset/DataFrame containing the same data in the stream.
Stream-static Joins
Since the introduction in Spark 2.0, Structured Streaming has supported joins (inner join and some type of outer joins) between a streaming and a
static DataFrame/Dataset. Here is a simple example.
Note that stream-static joins are not stateful, so no state management is necessary. However, a few types of stream-static outer joins are not yet
supported. These are listed at the end of this Join section.
Stream-stream Joins
In Spark 2.3, we have added support for stream-stream joins, that is, you can join two streaming Datasets/DataFrames. The challenge of
generating join results between two data streams is that, at any point of time, the view of the dataset is incomplete for both sides of the join
making it much harder to find matches between inputs. Any row received from one input stream can match with any future, yet-to-be-received
row from the other input stream. Hence, for both the input streams, we buffer past input as streaming state, so that we can match every future
input with past input and accordingly generate joined results. Furthermore, similar to streaming aggregations, we automatically handle late, out-
of-order data and can limit the state using watermarks. Let’s discuss the different types of supported stream-stream joins and how to use them.
1. Define watermark delays on both inputs such that the engine knows how delayed the input can be (similar to streaming aggregations)
2. Define a constraint on event-time across the two inputs such that the engine can figure out when old rows of one input is not going to be
required (i.e. will not satisfy the time constraint) for matches with the other input. This constraint can be defined in one of the two ways.
1. Time range join conditions (e.g. ...JOIN ON leftTime BETWEEN rightTime AND rightTime + INTERVAL 1 HOUR),
Let’s say we want to join a stream of advertisement impressions (when an ad was shown) with another stream of user clicks on advertisements to
correlate when impressions led to monetizable clicks. To allow the state cleanup in this stream-stream join, you will have to specify the
watermarking delays and the time constraints as follows.
1. Watermark delays: Say, the impressions and the corresponding clicks can be late/out-of-order in event-time by at most 2 and 3 hours,
respectively.
2. Event-time range condition: Say, a click can occur within a time range of 0 seconds to 1 hour after the corresponding impression.
import org.apache.spark.sql.functions.expr
impressionsWithWatermark.join(
clicksWithWatermark,
expr("""
clickAdId = impressionAdId AND
clickTime >= impressionTime AND
clickTime <= impressionTime + interval 1 hour
"""),
joinType = "leftOuter" // can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"
)
Caveats
There are a few important characteristics to note regarding how the outer results are generated.
The outer NULL results will be generated with a delay that depends on the specified watermark delay and the time range condition. This is
because the engine has to wait for that long to ensure there were no matches and there will be no more matches in future.
In the current implementation in the micro-batch engine, watermarks are advanced at the end of a micro-batch, and the next micro-batch
uses the updated watermark to clean up state and output outer results. Since we trigger a micro-batch only when there is new data to be
processed, the generation of the outer result may get delayed if there no new data being received in the stream. In short, if any of the two
input streams being joined does not receive data for a while, the outer (both cases, left or right) output may get delayed.
Static Static All types Supported, since its not on streaming data even though it can be present in a streaming query
Left
Supported, not stateful
Outer
Right
Not supported
Stream Static Outer
Full
Not supported
Outer
Left
Supported, not stateful
Semi
Left
Not supported
Outer
Right
Supported, not stateful
Static Stream Outer
Full
Not supported
Outer
Left
Not supported
Semi
Inner Supported, optionally specify watermark on both sides + time constraints for state cleanup
Left Conditionally supported, must specify watermark on right + time constraints for correct results,
Outer optionally specify watermark on left for all state cleanup
Right Conditionally supported, must specify watermark on left + time constraints for correct results,
Stream Stream Outer optionally specify watermark on right for all state cleanup
Full Conditionally supported, must specify watermark on one side + time constraints for correct results,
Outer optionally specify watermark on the other side for all state cleanup
Left Conditionally supported, must specify watermark on right + time constraints for correct results,
Semi optionally specify watermark on left for all state cleanup
Joins can be cascaded, that is, you can do df1.join(df2, ...).join(df3, ...).join(df4, ....).
As of Spark 2.4, you can use joins only when the query is in Append output mode. Other output modes are not yet supported.
As of Spark 2.4, you cannot use other non-map-like operations before joins. Here are a few examples of what cannot be used.
Streaming Deduplication
You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique
identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to
aggregations, you can use deduplication with or without watermarking.
With watermark - If there is an upper bound on how late a duplicate record may arrive, then you can define a watermark on an event time
column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from
past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain.
Without watermark - Since there are no bounds on when a duplicate record may arrive, the query stores the data from all the past records as
state.
Scala
While executing the query, Structured Streaming individually tracks the maximum event time seen in each input stream, calculates watermarks
based on the corresponding delay, and chooses a single global watermark with them to be used for stateful operations. By default, the minimum
is chosen as the global watermark because it ensures that no data is accidentally dropped as too late if one of the streams falls behind the others
(for example, one of the streams stops receiving data due to upstream failures). In other words, the global watermark will safely move at the pace
of the slowest stream and the query output will be delayed accordingly.
However, in some cases, you may want to get faster results even if it means dropping data from the slowest stream. Since Spark 2.4, you can set
the multiple watermark policy to choose the maximum value as the global watermark by setting the SQL configuration
spark.sql.streaming.multipleWatermarkPolicy to max (default is min). This lets the global watermark move at the pace of the fastest stream.
However, as a side effect, data from the slower streams will be aggressively dropped. Hence, use this configuration judiciously.
Though Spark cannot check and force it, the state function should be implemented with respect to the semantics of the output mode. For
example, in Update mode Spark doesn’t expect that the state function will emit rows which are older than current watermark plus allowed late
record delay, whereas in Append mode the state function can emit these rows.
Unsupported Operations
There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets. Some of them are as follows.
Limit and take the first N rows are not supported on streaming Datasets.
Sorting operations are supported on streaming Datasets only after an aggregation and in Complete Output Mode.
Few types of outer joins on streaming Datasets are not supported. See the support matrix in the Join Operations section for more details.
Chaining multiple stateful operations on streaming Datasets is not supported with Update and Complete mode.
In addition, below operations followed by other stateful operation is not supported in Append mode.
stream-stream time interval join (inner/outer)
flatMapGroupsWithState
A known workaround is to split your streaming query into multiple queries having a single stateful operation per each query, and ensure
end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional.
In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and
return results, which does not make sense on a streaming Dataset. Rather, those functionalities can be done by explicitly starting a streaming
query (see the next section regarding that).
count() - Cannot return a single count from a streaming Dataset. Instead, use ds.groupBy().count() which returns a streaming Dataset
containing a running count.
If you try any of these operations, you will see an AnalysisException like “operation XYZ is not supported with streaming DataFrames/Datasets”.
While some of them may be supported in future releases of Spark, there are others which are fundamentally hard to implement on streaming
data efficiently. For example, sorting on the input stream is not supported, as it requires keeping track of all the data received in the stream. This
is therefore fundamentally hard to execute efficiently.
State Store
State store is a versioned key-value store which provides both read and write operations. In Structured Streaming, we use the state store provider
to handle the stateful operations across batches. There are two built-in state store provider implementations. End users can also implement their
own state store provider by extending StateStoreProvider interface.
If you have stateful operations in your streaming query (for example, streaming aggregation, streaming dropDuplicates, stream-stream joins,
mapGroupsWithState, or flatMapGroupsWithState) and you want to maintain millions of keys in the state, then you may face issues related to
large JVM garbage collection (GC) pauses causing high variations in the micro-batch processing times. This occurs because, by the
implementation of HDFSBackedStateStore, the state data is maintained in the JVM memory of the executors and large number of state objects
puts memory pressure on the JVM causing high GC pauses.
In such cases, you can choose to use a more optimized state management solution based on RocksDB. Rather than keeping the state in the JVM
memory, this solution uses RocksDB to efficiently manage the state in the native memory and the local disk. Furthermore, any changes to this
state are automatically saved by Structured Streaming to the checkpoint location you have provided, thus providing full fault-tolerance
guarantees (the same as default state management).
Here are the configs regarding to RocksDB instance of the state store provider:
spark.sql.streaming.stateStore.rocksdb.lockAcquireTimeoutMs The waiting time in millisecond for acquiring lock in the 60000
load operation for RocksDB instance.
spark.sql.streaming.stateStore.rocksdb.maxOpenFiles The number of open files that can be used by the RocksDB -1
instance. Value of -1 means that files opened are always
kept open. If the open file limit is reached, RocksDB will
evict entries from the open file cache and close those file
descriptors and remove the entries from the cache.
spark.sql.streaming.stateStore.rocksdb.resetStatsOnLoad Whether we resets all ticker and histogram stats for True
RocksDB on load.
spark.sql.streaming.stateStore.rocksdb.trackTotalNumberOfRows Whether we track the total number of rows in state store. True
Please refer the details in Performance-aspect
considerations.
Performance-aspect considerations
1. You may want to disable the track of total number of rows to aim the better performance on RocksDB state store.
Tracking the number of rows brings additional lookup on write operations - you’re encouraged to try turning off the config on tuning RocksDB
state store, especially the values of metrics for state operator are big - numRowsUpdated, numRowsRemoved.
You can change the config during restarting the query, which enables you to change the trade-off decision on “observability vs performance”. If
the config is disabled, the number of rows in state (numTotalStateRows) will be reported as 0.
The stateful operations in Structured Streaming queries rely on the preferred location feature of Spark’s RDD to run the state store provider on
the same executor. If in the next batch the corresponding state store provider is scheduled on this executor again, it could reuse the previous
states and save the time of loading checkpointed states.
However, generally the preferred location is not a hard requirement and it is still possible that Spark schedules tasks to the executors other than
the preferred ones. In this case, Spark will load state store providers from checkpointed states on new executors. The state store providers run in
the previous batch will not be unloaded immediately. Spark runs a maintenance task which checks and unloads the state store providers that are
inactive on the executors.
By changing the Spark configurations related to task scheduling, for example spark.locality.wait, users can configure Spark how long to wait
to launch a data-local task. For stateful operations in Structured Streaming, it can be used to let state store providers running on the same
executors across batches.
Specifically for built-in HDFS state store provider, users can check the state store metrics such as loadedMapCacheHitCount and
loadedMapCacheMissCount. Ideally, it is best if cache missing count is minimized that means Spark won’t waste too much time on loading
checkpointed state. User can increase Spark locality waiting configurations to avoid loading state store providers in different executors across
batches.
Query name: Optionally, specify a unique name of the query for identification.
Trigger interval: Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the
previous processing has been completed. If a trigger time is missed because the previous processing has not been completed, then the
system will trigger processing immediately.
Checkpoint location: For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system
will write all the checkpoint information. This should be a directory in an HDFS-compatible fault-tolerant file system. The semantics of
checkpointing is discussed in more detail in the next section.
Output Modes
There are a few types of output modes.
Append mode (default) - This is the default mode, where only the new rows added to the Result Table since the last trigger will be
outputted to the sink. This is supported for only those queries where rows added to the Result Table is never going to change. Hence, this
mode guarantees that each row will be output only once (assuming fault-tolerant sink). For example, queries with only select, where, map,
flatMap, filter, join, etc. will support Append mode.
Complete mode - The whole Result Table will be outputted to the sink after every trigger. This is supported for aggregation queries.
Update mode - (Available since Spark 2.1.1) Only the rows in the Result Table that were updated since the last trigger will be outputted to the
sink. More information to be added in future releases.
Different types of streaming queries support different output modes. Here is the compatibility matrix.
Append mode uses watermark to drop old aggregation state. But the
output of a windowed aggregation is delayed the late threshold
specified in withWatermark() as by the modes semantics, rows can
be added to the Result Table only once after they are finalized (i.e.
Aggregation on Append,
after watermark is crossed). See the Late Data section for more details.
event-time with Update,
watermark Complete
Update mode uses watermark to drop old aggregation state.
Queries with mapGroupsWithState Update Aggregations not allowed in a query with mapGroupsWithState.
Append
Append Aggregations are allowed after flatMapGroupsWithState.
operation mode
Queries with
flatMapGroupsWithState
Update
Update Aggregations not allowed in a query with flatMapGroupsWithState.
operation mode
Update and Complete mode not supported yet. See the support
Queries with joins Append matrix in the Join Operations section for more details on what types
of joins are supported.
Output Sinks
There are a few types of built-in output sinks.
writeStream
.format("parquet") // can be "orc", "json", "csv", etc.
.option("path", "path/to/destination/dir")
.start()
writeStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "updates")
.start()
Foreach sink - Runs arbitrary computation on the records in the output. See later in the section for more details.
writeStream
.foreach(...)
.start()
Console sink (for debugging) - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output
modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the
driver’s memory after every trigger.
writeStream
.format("console")
.start()
Memory sink (for debugging) - The output is stored in memory as an in-memory table. Both, Append and Complete output modes, are
supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver’s
memory. Hence, use it with caution.
writeStream
.format("memory")
.queryName("tableName")
.start()
Some sinks are not fault-tolerant because they do not guarantee persistence of the output and are meant for debugging purposes only. See the
earlier section on fault-tolerance semantics. Here are the details of all the sinks in Spark.
File Sink Append path: path to the output directory, must be specified. Yes (exactly-once) Supports writes
retention: time to live (TTL) for output files. Output files to partitioned
which batches were committed older than TTL will be tables.
eventually excluded in metadata log. This means reader Partitioning by
queries which read the sink's output directory may not time may be
process them. You can provide the value as string format of useful.
the time. (like "12h", "7d", etc.) By default it's disabled.
Kafka Sink Append, See the Kafka Integration Guide Yes (at-least-once) More details in
Update, the Kafka
Complete Integration
Guide
Console Sink Append, numRows: Number of rows to print every trigger (default: No
Update, 20)
Complete truncate: Whether to truncate the output if too long
(default: true)
Memory Sink Append, None No. But in Complete Table name is the
Complete Mode, restarted query name.
query will recreate
the full table.
Note that you have to call start() to actually start the execution of the query. This returns a StreamingQuery object which is a handle to the
continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s
understand all this with a few examples.
ForeachBatch
foreachBatch(...) allows you to specify a function that is executed on the output data of every micro-batch of a streaming query. Since Spark
2.4, this is supported in Scala, Java and Python. It takes two parameters: a DataFrame or Dataset that has the output data of a micro-batch and
the unique ID of the micro-batch.
streamingDF.writeStream.foreachBatch(foreach_batch_function).start()
Reuse existing batch data sources - For many storage systems, there may not be a streaming sink available yet, but there may already exist
a data writer for batch queries. Using foreachBatch, you can use the batch data writers on the output of each micro-batch.
Write to multiple locations - If you want to write the output of a streaming query to multiple locations, then you can simply write the
output DataFrame/Dataset multiple times. However, each attempt to write can cause the output data to be recomputed (including possible
re-reading of the input data). To avoid recomputations, you should cache the output DataFrame/Dataset, write it to multiple locations, and
then uncache it. Here is an outline.
Scala
Apply additional DataFrame operations - Many DataFrame and Dataset operations are not supported in streaming DataFrames because
Spark does not support generating incremental plans in those cases. Using foreachBatch, you can apply some of these operations on each
micro-batch output. However, you will have to reason about the end-to-end semantics of doing that operation yourself.
Note:
By default, foreachBatch provides only at-least-once write guarantees. However, you can use the batchId provided to the function as way to
deduplicate the output and get an exactly-once guarantee.
foreachBatch does not work with the continuous processing mode as it fundamentally relies on the micro-batch execution of a streaming
query. If you write data in the continuous mode, use foreach instead.
Foreach
If foreachBatch is not an option (for example, corresponding batch data writer does not exist, or continuous processing mode), then you can
express your custom writer logic using foreach. Specifically, you can express the data writing logic by dividing it into three methods: open,
process, and close. Since Spark 2.4, foreach is available in Scala, Java and Python.
In Python, you can invoke foreach in two ways: in a function or in an object. The function offers a simple way to express your processing logic but
does not allow you to deduplicate generated data when failures cause reprocessing of some input data. For that situation you must specify the
processing logic in an object.
def process_row(row):
# Write row to storage
pass
query = streamingDF.writeStream.foreach(process_row).start()
Second, the object has a process method and optional open and close methods:
class ForeachWriter:
def open(self, partition_id, epoch_id):
# Open connection. This method is optional in Python.
pass
query = streamingDF.writeStream.foreach(ForeachWriter()).start()
Execution semantics When the streaming query is started, Spark calls the function or the object’s methods in the following way:
A single copy of this object is responsible for all the data generated by a single task in a query. In other words, one instance is responsible for
processing one partition of the data generated in a distributed manner.
This object must be serializable, because each task will get a fresh serialized-deserialized copy of the provided object. Hence, it is strongly
recommended that any initialization for writing data (for example. opening a connection or starting a transaction) is done after the open()
method has been called, which signifies that the task is ready to generate data.
If open(…) returns true, for each row in the partition and batch/epoch, method process(row) is called.
Method close(error) is called with error (if any) seen while processing rows.
The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM
or Python process crashes in the middle.
Note: Spark does not guarantee same output for (partitionId, epochId), so deduplication cannot be achieved with (partitionId, epochId). e.g.
source provides different number of partitions for some reasons, Spark optimization changes number of partitions, etc. See SPARK-28650 for
more details. If you need deduplication on output, try out foreachBatch instead.
For more details, please check the docs for DataStreamReader (Scala/Java/Python docs) and DataStreamWriter (Scala/Java/Python docs).
Triggers
The trigger settings of a streaming query define the timing of streaming data processing, whether the query is going to be executed as micro-
batch query with a fixed batch interval or as a continuous processing query. Here are the different kinds of triggers that are supported.
unspecified (default) If no trigger setting is explicitly specified, then by default, the query will be executed in micro-batch mode, where
micro-batches will be generated as soon as the previous micro-batch has completed processing.
Fixed interval micro- The query will be executed with micro-batches mode, where micro-batches will be kicked off at the user-specified
batches intervals.
If the previous micro-batch completes within the interval, then the engine will wait until the interval is over
before kicking off the next micro-batch.
If the previous micro-batch takes longer than the interval to complete (i.e. if an interval boundary is missed),
then the next micro-batch will start as soon as the previous one completes (i.e., it will not wait for the next
interval boundary).
If no new data is available, then no micro-batch will be kicked off.
One-time micro- The query will execute only one micro-batch to process all the available data and then stop on its own. This is
batch(deprecated) useful in scenarios you want to periodically spin up a cluster, process everything that is available since the last
period, and then shutdown the cluster. In some case, this may lead to significant cost savings. Note that this trigger
is deprecated and users are encouraged to migrate to Available-now micro-batch, as it provides the better
guarantee of processing, fine-grained scale of batches, and better gradual processing of watermark advancement
including no-data batch.
Available-now Similar to queries one-time micro-batch trigger, the query will process all the available data and then stop on its
micro-batch own. The difference is that, it will process the data in (possibly) multiple micro-batches based on the source
options (e.g. maxFilesPerTrigger for file source), which will result in better query scalability.
This trigger provides a strong guarantee of processing: regardless of how many batches were left over in
previous run, it ensures all available data at the time of execution gets processed before termination. All
uncommitted batches will be processed first.
Watermark gets advanced per each batch, and no-data batch gets executed before termination if the last
batch advances the watermark. This helps to maintain smaller and predictable state size and smaller latency on
the output of stateful operators.
Continuous with The query will be executed in the new low-latency, continuous processing mode. Read more about this in the
fixed checkpoint Continuous Processing section below.
interval
(experimental)
import org.apache.spark.sql.streaming.Trigger
// Available-now trigger
df.writeStream
.format("console")
.trigger(Trigger.AvailableNow())
.start()
query.id // get the unique identifier of the running query that persists across restarts from checkpoint data
query.runId // get the unique id of this run of the query, which will be generated at every start/restart
query.exception // the exception if the query has been terminated with error
query.recentProgress // an array of the most recent progress updates for this query
You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use
sparkSession.streams() to get the StreamingQueryManager (Scala/Java/Python docs) that can be used to manage the currently active queries.
In addition, streamingQuery.status() returns a StreamingQueryStatus object in Scala and Java and a dictionary with the same fields in Python.
It gives information about what the query is immediately doing - is a trigger active, is data being processed, etc.
println(query.lastProgress)
{
"id" : "ce011fdc-8762-4dcb-84eb-a77333e28109",
"runId" : "88e2ff94-ede0-45a8-b687-6316fbef529a",
"name" : "MyQuery",
"timestamp" : "2016-12-14T18:45:24.873Z",
"numInputRows" : 10,
"inputRowsPerSecond" : 120.0,
"processedRowsPerSecond" : 200.0,
"durationMs" : {
"triggerExecution" : 3,
"getOffset" : 2
},
"eventTime" : {
"watermark" : "2016-12-14T18:45:24.873Z"
},
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-0]]",
"startOffset" : {
"topic-0" : {
"2" : 0,
"4" : 1,
"1" : 1,
"3" : 1,
"0" : 1
}
},
"endOffset" : {
"topic-0" : {
"2" : 0,
"4" : 115,
"1" : 134,
"3" : 21,
"0" : 534
}
},
"numInputRows" : 10,
"inputRowsPerSecond" : 120.0,
"processedRowsPerSecond" : 200.0
} ],
"sink" : {
"description" : "MemorySink"
}
}
*/
println(query.status)
spark.streams.addListener(new StreamingQueryListener() {
override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = {
println("Query started: " + queryStarted.id)
}
override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = {
println("Query terminated: " + queryTerminated.id)
}
override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = {
println("Query made progress: " + queryProgress.progress)
}
})
spark.conf.set("spark.sql.streaming.metricsEnabled", "true")
// or
spark.sql("SET spark.sql.streaming.metricsEnabled=true")
All queries started in the SparkSession after this configuration has been enabled will report metrics through Dropwizard to whatever sinks have
been configured (e.g. Ganglia, Graphite, JMX, etc.).
aggDF
.writeStream
.outputMode("complete")
.option("checkpointLocation", "path/to/HDFS/dir")
.format("memory")
.start()
The term allowed means you can do the specified change but whether the semantics of its effect is well-defined depends on the query and
the change.
The term not allowed means you should not do the specified change as the restarted query is likely to fail with unpredictable errors. sdf
represents a streaming DataFrame/Dataset generated with sparkSession.readStream.
Types of changes
Changes in the number or type (i.e. different source) of input sources: This is not allowed.
Changes in the parameters of input sources: Whether this is allowed and whether the semantics of the change are well-defined depends on
the source and the query. Here are a few examples.
Changes to subscribed topics/files are generally not allowed as the results are unpredictable:
spark.readStream.format("kafka").option("subscribe", "topic") to
spark.readStream.format("kafka").option("subscribe", "newTopic")
Changes in the type of output sink: Changes between a few specific combinations of sinks are allowed. This needs to be verified on a case-by-
case basis. Here are a few examples.
File sink to Kafka sink is allowed. Kafka will see only the new data.
Changes in the parameters of output sink: Whether this is allowed and whether the semantics of the change are well-defined depends on the
sink and the query. Here are a few examples.
Changes to output directory of a file sink are not allowed: sdf.writeStream.format("parquet").option("path", "/somePath") to
sdf.writeStream.format("parquet").option("path", "/anotherPath")
Changes to the user-defined foreach sink (that is, the ForeachWriter code) are allowed, but the semantics of the change depends on the
code.
Changes in projection / filter / map-like operations: Some cases are allowed. For example:
Changes in projections with same output schema are allowed: sdf.selectExpr("stringColumn AS json").writeStream to
sdf.selectExpr("anotherStringColumn AS json").writeStream
Changes in projections with different output schema are conditionally allowed: sdf.selectExpr("a").writeStream to
sdf.selectExpr("b").writeStream is allowed only if the output sink allows the schema change from "a" to "b".
Changes in stateful operations: Some operations in streaming queries need to maintain state data in order to continuously update the result.
Structured Streaming automatically checkpoints the state data to fault-tolerant storage (for example, HDFS, AWS S3, Azure Blob storage) and
restores it after restart. However, this assumes that the schema of the state data remains same across restarts. This means that any changes
(that is, additions, deletions, or schema modifications) to the stateful operations of a streaming query are not allowed between restarts. Here is
the list of stateful operations whose schema should not be changed between restarts in order to ensure state recovery:
Streaming aggregation: For example, sdf.groupBy("a").agg(...). Any change in number or type of grouping keys or aggregates is not
allowed.
Streaming deduplication: For example, sdf.dropDuplicates("a"). Any change in number or type of deduplicating columns is not
allowed.
Stream-stream join: For example, sdf1.join(sdf2, ...) (i.e. both inputs are generated with sparkSession.readStream). Changes in
the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) are not allowed. Other changes in the join
condition are ill-defined.
The table below describes the configurations for this feature and default values associated with them.
asyncProgressCheckpointingInterval minutes 1 the interval in which we commit offsets and completion commits
Limitations
The initial version of the feature has the following limitations:
Asynchronous progress tracking is only supported in stateless queries using Kafka Sink
Exactly once end-to-end processing will not be supported with this asynchronous progress tracking because offset ranges for batch can be
changed in case of failure. Though many sinks, such as Kafka sink, do not support writing exactly once anyways.
Also the following error message may be printed in the driver logs:
The offset log for batch x doesn't exist, which is required to restart the query from the latest batch x from the offset lo
This is caused by the fact that when async progress tracking is enabled, the framework will not checkpoint progress for every batch as would be
done if async progress tracking is not used. To solve this problem simply re-enable “asyncProgressTrackingEnabled” and set
“asyncProgressCheckpointingInterval” to 0 and run the streaming query until at least two micro-batches have been processed. Async progress
tracking can be now safely disabled and restarting query should proceed normally.
Continuous Processing
[Experimental]
Continuous processing is a new, experimental streaming execution mode introduced in Spark 2.3 that enables low (~1 ms) end-to-end latency
with at-least-once fault-tolerance guarantees. Compare this with the default micro-batch processing engine which can achieve exactly-once
guarantees but achieve latencies of ~100ms at best. For some types of queries (discussed below), you can choose which mode to execute them in
without modifying the application logic (i.e. without changing the DataFrame/Dataset operations).
To run a supported query in continuous processing mode, all you need to do is specify a continuous trigger with the desired checkpoint interval
as a parameter. For example,
import org.apache.spark.sql.streaming.Trigger
spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.load()
.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.writeStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "topic1")
.trigger(Trigger.Continuous("1 second")) // only change in query
.start()
A checkpoint interval of 1 second means that the continuous processing engine will record the progress of the query every second. The resulting
checkpoints are in a format compatible with the micro-batch engine, hence any query can be restarted with any trigger. For example, a supported
query started with the micro-batch mode can be restarted in continuous mode, and vice versa. Note that any time you switch to continuous
mode, you will get at-least-once fault-tolerance guarantees.
Supported Queries
As of Spark 2.4, only the following type of queries are supported in the continuous processing mode.
Operations: Only map-like Dataset/DataFrame operations are supported in continuous mode, that is, only projections (select, map, flatMap,
mapPartitions, etc.) and selections (where, filter, etc.).
All SQL functions are supported except aggregation functions (since aggregations are not yet supported), current_timestamp() and
current_date() (deterministic computations using time is challenging).
Sources:
Kafka source: All options are supported.
Rate source: Good for testing. Only options that are supported in the continuous mode are numPartitions and rowsPerSecond.
Sinks:
Kafka sink: All options are supported.
Memory sink: Good for debugging.
Console sink: Good for debugging. All options are supported. Note that the console will print every checkpoint interval that you have
specified in the continuous trigger.
See Input Sources and Output Sinks sections for more details on them. While the console sink is good for testing, the end-to-end low-latency
processing can be best observed with Kafka as the source and sink, as this allows the engine to process the data and make the results available in
the output topic within milliseconds of the input data being available in the input topic.
Caveats
Continuous processing engine launches multiple long-running tasks that continuously read data from sources, process it and continuously
write to sinks. The number of tasks required by the query depends on how many partitions the query can read from the sources in parallel.
Therefore, before starting a continuous processing query, you must ensure there are enough cores in the cluster to all the tasks in parallel. For
example, if you are reading from a Kafka topic that has 10 partitions, then the cluster must have at least 10 cores for the query to make
progress.
Stopping a continuous processing stream may produce spurious task termination warnings. These can be safely ignored.
There are currently no automatic retries of failed tasks. Any failure will lead to the query being stopped and it needs to be manually restarted
from the checkpoint.
Additional Information
Notes
Several configurations are not modifiable after the query has run. To change them, discard the checkpoint and start a new query. These
configurations include:
spark.sql.shuffle.partitions
This is due to the physical partitioning of state: state is partitioned via applying hash function to key, hence the number of partitions
for state should be unchanged.
If you want to run fewer tasks for stateful operations, coalesce would help with avoiding unnecessary repartitioning.
After coalesce, the number of (reduced) tasks will be kept unless another shuffle happens.
spark.sql.streaming.stateStore.providerClass: To read the previous state of the query properly, the class of state store provider
should be unchanged.
spark.sql.streaming.multipleWatermarkPolicy: Modification of this would lead inconsistent watermark value when query contains
multiple watermarks, hence the policy should be unchanged.
Further Reading
Talks
Migration Guide
The migration guide is now archived on this page.