blob: 91f2d510e6a843d9e86bd04d9b475afae92ced0d [file] [log] [blame]
<!DOCTYPE html>
<html class="no-js">
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>Structured Streaming Programming Guide - Spark 4.1.0-preview1 Documentation</title>
<link rel="stylesheet" href="../css/bootstrap.min.css">
<link rel="preconnect" href="https://fonts.googleapis.com">
<link rel="preconnect" href="https://fonts.gstatic.com" crossorigin>
<link href="https://fonts.googleapis.com/css2?family=DM+Sans:ital,wght@0,400;0,500;0,700;1,400;1,500;1,700&Courier+Prime:wght@400;700&display=swap" rel="stylesheet">
<link href="../css/custom.css" rel="stylesheet">
<script src="../js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="../css/pygments-default.css">
<link rel="stylesheet" href="../css/docsearch.min.css" />
<link rel="stylesheet" href="../css/docsearch.css">
<!-- Matomo -->
<script>
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
_paq.push(["disableCookies"]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="https://analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '40']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
<!-- End Matomo Code -->
</head>
<body class="global">
<!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
<nav class="navbar navbar-expand-lg navbar-dark p-0 px-4 fixed-top" style="background: #1d6890;" id="topbar">
<div class="navbar-brand"><a href="../index.html">
<img src="https://spark.apache.org/images/spark-logo-rev.svg" width="141" height="72"/></a><span class="version">4.1.0-preview1</span>
</div>
<button class="navbar-toggler" type="button" data-toggle="collapse"
data-target="#navbarCollapse" aria-controls="navbarCollapse"
aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div class="collapse navbar-collapse" id="navbarCollapse">
<ul class="navbar-nav me-auto">
<li class="nav-item"><a href="../index.html" class="nav-link">Overview</a></li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarQuickStart" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Programming Guides</a>
<div class="dropdown-menu" aria-labelledby="navbarQuickStart">
<a class="dropdown-item" href="../quick-start.html">Quick Start</a>
<a class="dropdown-item" href="../rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a>
<a class="dropdown-item" href="../sql-programming-guide.html">SQL, DataFrames, and Datasets</a>
<a class="dropdown-item" href="../streaming/index.html">Structured Streaming</a>
<a class="dropdown-item" href="../streaming-programming-guide.html">Spark Streaming (DStreams)</a>
<a class="dropdown-item" href="../ml-guide.html">MLlib (Machine Learning)</a>
<a class="dropdown-item" href="../graphx-programming-guide.html">GraphX (Graph Processing)</a>
<a class="dropdown-item" href="../sparkr.html">SparkR (R on Spark)</a>
<a class="dropdown-item" href="../api/python/getting_started/index.html">PySpark (Python on Spark)</a>
<a class="dropdown-item" href="../declarative-pipelines-programming-guide.html">Declarative Pipelines</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarAPIDocs" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">API Docs</a>
<div class="dropdown-menu" aria-labelledby="navbarAPIDocs">
<a class="dropdown-item" href="../api/python/index.html">Python</a>
<a class="dropdown-item" href="../api/scala/org/apache/spark/index.html">Scala</a>
<a class="dropdown-item" href="../api/java/index.html">Java</a>
<a class="dropdown-item" href="../api/R/index.html">R</a>
<a class="dropdown-item" href="../api/sql/index.html">SQL, Built-in Functions</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarDeploying" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Deploying</a>
<div class="dropdown-menu" aria-labelledby="navbarDeploying">
<a class="dropdown-item" href="../cluster-overview.html">Overview</a>
<a class="dropdown-item" href="../submitting-applications.html">Submitting Applications</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="../spark-standalone.html">Spark Standalone</a>
<a class="dropdown-item" href="../running-on-yarn.html">YARN</a>
<a class="dropdown-item" href="../running-on-kubernetes.html">Kubernetes</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarMore" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">More</a>
<div class="dropdown-menu" aria-labelledby="navbarMore">
<a class="dropdown-item" href="../configuration.html">Configuration</a>
<a class="dropdown-item" href="../monitoring.html">Monitoring</a>
<a class="dropdown-item" href="../tuning.html">Tuning Guide</a>
<a class="dropdown-item" href="../job-scheduling.html">Job Scheduling</a>
<a class="dropdown-item" href="../security.html">Security</a>
<a class="dropdown-item" href="../hardware-provisioning.html">Hardware Provisioning</a>
<a class="dropdown-item" href="../migration-guide.html">Migration Guide</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="../building-spark.html">Building Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/contributing.html">Contributing to Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a>
</div>
</li>
<li class="nav-item">
<input type="text" id="docsearch-input" placeholder="Search the docs…">
</li>
</ul>
<!--<span class="navbar-text navbar-right"><span class="version-text">v4.1.0-preview1</span></span>-->
</div>
</nav>
<div class="container">
<div class="left-menu-wrapper">
<div class="left-menu">
<h3><a href="../streaming/index.html">Structured Streaming Programming Guide</a></h3>
<ul>
<li>
<a href="../streaming/index.html">
Overview
</a>
</li>
<li>
<a href="../streaming/getting-started.html">
Getting Started
</a>
</li>
<ul>
<li>
<a href="../streaming/getting-started.html#quick-example">
Quick Example
</a>
</li>
<li>
<a href="../streaming/getting-started.html#programming-model">
Programming Model
</a>
</li>
</ul>
<li>
<a href="../streaming/apis-on-dataframes-and-datasets.html">
APIs on DataFrames and Datasets
</a>
</li>
<li>
<a href="../streaming/performance-tips.html">
Performance Tips
</a>
</li>
<li>
<a href="../streaming/additional-information.html">
Additional Information
</a>
</li>
</ul>
</div>
</div>
<input id="nav-trigger" class="nav-trigger" checked type="checkbox">
<label for="nav-trigger"></label>
<div class="content-with-sidebar mr-3" id="content">
<h1 class="title">Structured Streaming Programming Guide</h1>
<h1 id="quick-example">Quick Example</h1>
<p>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
<a href="https://github.com/apache/spark/blob/v4.1.0-preview1/examples/src/main/python/sql/streaming/structured_network_wordcount.py">Python</a>/<a href="https://github.com/apache/spark/blob/v4.1.0-preview1/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala">Scala</a>/<a href="https://github.com/apache/spark/blob/v4.1.0-preview1/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java">Java</a>/<a href="https://github.com/apache/spark/blob/v4.1.0-preview1/examples/src/main/r/streaming/structured_network_wordcount.R">R</a>.
And if you <a href="https://spark.apache.org/downloads.html">download Spark</a>, you can directly <a href="../index.html#running-the-examples-and-shell">run the example</a>. 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.</p>
<div class="codetabs">
<div data-lang="python">
<figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="kn">from</span> <span class="n">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span>
<span class="kn">from</span> <span class="n">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">explode</span>
<span class="kn">from</span> <span class="n">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">split</span>
<span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span> \
<span class="p">.</span><span class="n">builder</span> \
<span class="p">.</span><span class="nf">appName</span><span class="p">(</span><span class="sh">"</span><span class="s">StructuredNetworkWordCount</span><span class="sh">"</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">getOrCreate</span><span class="p">()</span></code></pre></figure>
</div>
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">import</span> <span class="nn">org.apache.spark.sql.functions._</span>
<span class="k">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span>
<span class="k">val</span> <span class="nv">spark</span> <span class="k">=</span> <span class="nc">SparkSession</span>
<span class="o">.</span><span class="py">builder</span>
<span class="o">.</span><span class="py">appName</span><span class="o">(</span><span class="s">"StructuredNetworkWordCount"</span><span class="o">)</span>
<span class="o">.</span><span class="py">getOrCreate</span><span class="o">()</span>
<span class="k">import</span> <span class="nn">spark.implicits._</span></code></pre></figure>
</div>
<div data-lang="java">
<figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.FlatMapFunction</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.spark.sql.*</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.spark.sql.streaming.StreamingQuery</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">java.util.Arrays</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">java.util.Iterator</span><span class="o">;</span>
<span class="nc">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="nc">SparkSession</span>
<span class="o">.</span><span class="na">builder</span><span class="o">()</span>
<span class="o">.</span><span class="na">appName</span><span class="o">(</span><span class="s">"JavaStructuredNetworkWordCount"</span><span class="o">)</span>
<span class="o">.</span><span class="na">getOrCreate</span><span class="o">();</span></code></pre></figure>
</div>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">sparkR.session</span><span class="p">(</span><span class="n">appName</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"StructuredNetworkWordCount"</span><span class="p">)</span></code></pre></figure>
</div>
</div>
<p>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.</p>
<div class="codetabs">
<div data-lang="python">
<figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="c1"># Create DataFrame representing the stream of input lines from connection to localhost:9999
</span><span class="n">lines</span> <span class="o">=</span> <span class="n">spark</span> \
<span class="p">.</span><span class="n">readStream</span> \
<span class="p">.</span><span class="nf">format</span><span class="p">(</span><span class="sh">"</span><span class="s">socket</span><span class="sh">"</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">option</span><span class="p">(</span><span class="sh">"</span><span class="s">host</span><span class="sh">"</span><span class="p">,</span> <span class="sh">"</span><span class="s">localhost</span><span class="sh">"</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">option</span><span class="p">(</span><span class="sh">"</span><span class="s">port</span><span class="sh">"</span><span class="p">,</span> <span class="mi">9999</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">load</span><span class="p">()</span>
<span class="c1"># Split the lines into words
</span><span class="n">words</span> <span class="o">=</span> <span class="n">lines</span><span class="p">.</span><span class="nf">select</span><span class="p">(</span>
<span class="nf">explode</span><span class="p">(</span>
<span class="nf">split</span><span class="p">(</span><span class="n">lines</span><span class="p">.</span><span class="n">value</span><span class="p">,</span> <span class="sh">"</span><span class="s"> </span><span class="sh">"</span><span class="p">)</span>
<span class="p">).</span><span class="nf">alias</span><span class="p">(</span><span class="sh">"</span><span class="s">word</span><span class="sh">"</span><span class="p">)</span>
<span class="p">)</span>
<span class="c1"># Generate running word count
</span><span class="n">wordCounts</span> <span class="o">=</span> <span class="n">words</span><span class="p">.</span><span class="nf">groupBy</span><span class="p">(</span><span class="sh">"</span><span class="s">word</span><span class="sh">"</span><span class="p">).</span><span class="nf">count</span><span class="p">()</span></code></pre></figure>
<p>This <code class="language-plaintext highlighter-rouge">lines</code> DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named &#8220;value&#8221;, 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 <code class="language-plaintext highlighter-rouge">alias</code> to name the new column as &#8220;word&#8221;. Finally, we have defined the <code class="language-plaintext highlighter-rouge">wordCounts</code> 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.</p>
</div>
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Create DataFrame representing the stream of input lines from connection to localhost:9999</span>
<span class="k">val</span> <span class="nv">lines</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">readStream</span>
<span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"socket"</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"host"</span><span class="o">,</span> <span class="s">"localhost"</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"port"</span><span class="o">,</span> <span class="mi">9999</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="c1">// Split the lines into words</span>
<span class="k">val</span> <span class="nv">words</span> <span class="k">=</span> <span class="nv">lines</span><span class="o">.</span><span class="py">as</span><span class="o">[</span><span class="kt">String</span><span class="o">].</span><span class="py">flatMap</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">split</span><span class="o">(</span><span class="s">" "</span><span class="o">))</span>
<span class="c1">// Generate running word count</span>
<span class="k">val</span> <span class="nv">wordCounts</span> <span class="k">=</span> <span class="nv">words</span><span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="s">"value"</span><span class="o">).</span><span class="py">count</span><span class="o">()</span></code></pre></figure>
<p>This <code class="language-plaintext highlighter-rouge">lines</code> DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named &#8220;value&#8221;, 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 <code class="language-plaintext highlighter-rouge">.as[String]</code>, so that we can apply the <code class="language-plaintext highlighter-rouge">flatMap</code> operation to split each line into multiple words. The resultant <code class="language-plaintext highlighter-rouge">words</code> Dataset contains all the words. Finally, we have defined the <code class="language-plaintext highlighter-rouge">wordCounts</code> 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.</p>
</div>
<div data-lang="java">
<figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// Create DataFrame representing the stream of input lines from connection to localhost:9999</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</span> <span class="n">lines</span> <span class="o">=</span> <span class="n">spark</span>
<span class="o">.</span><span class="na">readStream</span><span class="o">()</span>
<span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"socket"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"host"</span><span class="o">,</span> <span class="s">"localhost"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"port"</span><span class="o">,</span> <span class="mi">9999</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">();</span>
<span class="c1">// Split the lines into words</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">String</span><span class="o">&gt;</span> <span class="n">words</span> <span class="o">=</span> <span class="n">lines</span>
<span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="nc">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">())</span>
<span class="o">.</span><span class="na">flatMap</span><span class="o">((</span><span class="nc">FlatMapFunction</span><span class="o">&lt;</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">&gt;)</span> <span class="n">x</span> <span class="o">-&gt;</span> <span class="nc">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span><span class="n">x</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">" "</span><span class="o">)).</span><span class="na">iterator</span><span class="o">(),</span> <span class="nc">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span>
<span class="c1">// Generate running word count</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">words</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"value"</span><span class="o">).</span><span class="na">count</span><span class="o">();</span></code></pre></figure>
<p>This <code class="language-plaintext highlighter-rouge">lines</code> DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named &#8220;value&#8221;, 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 <code class="language-plaintext highlighter-rouge">.as(Encoders.STRING())</code>, so that we can apply the <code class="language-plaintext highlighter-rouge">flatMap</code> operation to split each line into multiple words. The resultant <code class="language-plaintext highlighter-rouge">words</code> Dataset contains all the words. Finally, we have defined the <code class="language-plaintext highlighter-rouge">wordCounts</code> 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.</p>
</div>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Create DataFrame representing the stream of input lines from connection to localhost:9999</span><span class="w">
</span><span class="n">lines</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="s2">"socket"</span><span class="p">,</span><span class="w"> </span><span class="n">host</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"localhost"</span><span class="p">,</span><span class="w"> </span><span class="n">port</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="m">9999</span><span class="p">)</span><span class="w">
</span><span class="c1"># Split the lines into words</span><span class="w">
</span><span class="n">words</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">selectExpr</span><span class="p">(</span><span class="n">lines</span><span class="p">,</span><span class="w"> </span><span class="s2">"explode(split(value, ' ')) as word"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Generate running word count</span><span class="w">
</span><span class="n">wordCounts</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">count</span><span class="p">(</span><span class="n">group_by</span><span class="p">(</span><span class="n">words</span><span class="p">,</span><span class="w"> </span><span class="s2">"word"</span><span class="p">))</span></code></pre></figure>
<p>This <code class="language-plaintext highlighter-rouge">lines</code> SparkDataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named &#8220;value&#8221;, 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 a SQL expression with two SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we name the new column as &#8220;word&#8221;. Finally, we have defined the <code class="language-plaintext highlighter-rouge">wordCounts</code> SparkDataFrame by grouping by the unique values in the SparkDataFrame and counting them. Note that this is a streaming SparkDataFrame which represents the running word counts of the stream.</p>
</div>
</div>
<p>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 <code class="language-plaintext highlighter-rouge">outputMode("complete")</code>) to the console every time they are updated. And then start the streaming computation using <code class="language-plaintext highlighter-rouge">start()</code>.</p>
<div class="codetabs">
<div data-lang="python">
<figure class="highlight"><pre><code class="language-python" data-lang="python"> <span class="c1"># Start running the query that prints the running counts to the console
</span><span class="n">query</span> <span class="o">=</span> <span class="n">wordCounts</span> \
<span class="p">.</span><span class="n">writeStream</span> \
<span class="p">.</span><span class="nf">outputMode</span><span class="p">(</span><span class="sh">"</span><span class="s">complete</span><span class="sh">"</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">format</span><span class="p">(</span><span class="sh">"</span><span class="s">console</span><span class="sh">"</span><span class="p">)</span> \
<span class="p">.</span><span class="nf">start</span><span class="p">()</span>
<span class="n">query</span><span class="p">.</span><span class="nf">awaitTermination</span><span class="p">()</span></code></pre></figure>
</div>
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Start running the query that prints the running counts to the console</span>
<span class="k">val</span> <span class="nv">query</span> <span class="k">=</span> <span class="nv">wordCounts</span><span class="o">.</span><span class="py">writeStream</span>
<span class="o">.</span><span class="py">outputMode</span><span class="o">(</span><span class="s">"complete"</span><span class="o">)</span>
<span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"console"</span><span class="o">)</span>
<span class="o">.</span><span class="py">start</span><span class="o">()</span>
<span class="nv">query</span><span class="o">.</span><span class="py">awaitTermination</span><span class="o">()</span></code></pre></figure>
</div>
<div data-lang="java">
<figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// Start running the query that prints the running counts to the console</span>
<span class="nc">StreamingQuery</span> <span class="n">query</span> <span class="o">=</span> <span class="n">wordCounts</span><span class="o">.</span><span class="na">writeStream</span><span class="o">()</span>
<span class="o">.</span><span class="na">outputMode</span><span class="o">(</span><span class="s">"complete"</span><span class="o">)</span>
<span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"console"</span><span class="o">)</span>
<span class="o">.</span><span class="na">start</span><span class="o">();</span>
<span class="n">query</span><span class="o">.</span><span class="na">awaitTermination</span><span class="o">();</span></code></pre></figure>
</div>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Start running the query that prints the running counts to the console</span><span class="w">
</span><span class="n">query</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">write.stream</span><span class="p">(</span><span class="n">wordCounts</span><span class="p">,</span><span class="w"> </span><span class="s2">"console"</span><span class="p">,</span><span class="w"> </span><span class="n">outputMode</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"complete"</span><span class="p">)</span><span class="w">
</span><span class="n">awaitTermination</span><span class="p">(</span><span class="n">query</span><span class="p">)</span></code></pre></figure>
</div>
</div>
<p>After this code is executed, the streaming computation will have started in the background. The <code class="language-plaintext highlighter-rouge">query</code> object is a handle to that active streaming query, and we have decided to wait for the termination of the query using <code class="language-plaintext highlighter-rouge">awaitTermination()</code> to prevent the process from exiting while the query is active.</p>
<p>To actually execute this example code, you can either compile the code in your own
<a href="../quick-start.html#self-contained-applications">Spark application</a>, or simply
<a href="../index.html#running-the-examples-and-shell">run the example</a> 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</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ nc -lk 9999
</code></pre></div></div>
<p>Then, in a different terminal, you can start the example by using</p>
<div class="codetabs">
<div data-lang="python">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nv">$ </span>./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999</code></pre></figure>
</div>
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nv">$ </span>./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999</code></pre></figure>
</div>
<div data-lang="java">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nv">$ </span>./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999</code></pre></figure>
</div>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nv">$ </span>./bin/spark-submit examples/src/main/r/streaming/structured_network_wordcount.R localhost 9999</code></pre></figure>
</div>
</div>
<p>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.</p>
<table width="100%">
<td>
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="c"># TERMINAL 1:</span>
<span class="c"># Running Netcat</span>
<span class="nv">$ </span>nc <span class="nt">-lk</span> 9999
apache spark
apache hadoop
...</code></pre></figure>
</td>
<td width="2%"></td>
<td>
<div class="codetabs">
<div data-lang="python">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="c"># TERMINAL 2: RUNNING structured_network_wordcount.py</span>
<span class="nv">$ </span>./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999
<span class="nt">-------------------------------------------</span>
Batch: 0
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 1|
| spark| 1|
+------+-----+
<span class="nt">-------------------------------------------</span>
Batch: 1
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 2|
| spark| 1|
|hadoop| 1|
+------+-----+
...</code></pre></figure>
</div>
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="c"># TERMINAL 2: RUNNING StructuredNetworkWordCount</span>
<span class="nv">$ </span>./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999
<span class="nt">-------------------------------------------</span>
Batch: 0
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 1|
| spark| 1|
+------+-----+
<span class="nt">-------------------------------------------</span>
Batch: 1
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 2|
| spark| 1|
|hadoop| 1|
+------+-----+
...</code></pre></figure>
</div>
<div data-lang="java">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="c"># TERMINAL 2: RUNNING JavaStructuredNetworkWordCount</span>
<span class="nv">$ </span>./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999
<span class="nt">-------------------------------------------</span>
Batch: 0
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 1|
| spark| 1|
+------+-----+
<span class="nt">-------------------------------------------</span>
Batch: 1
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 2|
| spark| 1|
|hadoop| 1|
+------+-----+
...</code></pre></figure>
</div>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="c"># TERMINAL 2: RUNNING structured_network_wordcount.R</span>
<span class="nv">$ </span>./bin/spark-submit examples/src/main/r/streaming/structured_network_wordcount.R localhost 9999
<span class="nt">-------------------------------------------</span>
Batch: 0
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 1|
| spark| 1|
+------+-----+
<span class="nt">-------------------------------------------</span>
Batch: 1
<span class="nt">-------------------------------------------</span>
+------+-----+
| value|count|
+------+-----+
|apache| 2|
| spark| 1|
|hadoop| 1|
+------+-----+
...</code></pre></figure>
</div>
</div>
</td>
</table>
<h1 id="programming-model">Programming Model</h1>
<p>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 <em>incremental</em> query on the <em>unbounded</em> input
table. Let’s understand this model in more detail.</p>
<h2 id="basic-concepts">Basic Concepts</h2>
<p>Consider the input data stream as the &#8220;Input Table&#8221;. Every data item that is
arriving on the stream is like a new row being appended to the Input Table.</p>
<p><img src="../img/structured-streaming-stream-as-a-table.png" alt="Stream as a Table" title="Stream as a Table" /></p>
<p>A query on the input will generate the &#8220;Result Table&#8221;. 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.</p>
<p><img src="../img/structured-streaming-model.png" alt="Model" /></p>
<p>The &#8220;Output&#8221; is defined as what gets written out to the external storage. The output can be defined in a different mode:</p>
<ul>
<li>
<p><em>Complete Mode</em> - 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.</p>
</li>
<li>
<p><em>Append Mode</em> - 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.</p>
</li>
<li>
<p><em>Update Mode</em> - 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&#8217;t contain aggregations, it will be equivalent to Append mode.</p>
</li>
</ul>
<p>Note that each mode is applicable on certain types of queries. This is discussed in detail <a href="./apis-on-dataframes-and-datasets.html#output-modes">later</a>.</p>
<p>To illustrate the use of this model, let’s understand the model in context of
the <a href="#quick-example">Quick Example</a> above. The first <code class="language-plaintext highlighter-rouge">lines</code> DataFrame is the input table, and
the final <code class="language-plaintext highlighter-rouge">wordCounts</code> DataFrame is the result table. Note that the query on
streaming <code class="language-plaintext highlighter-rouge">lines</code> DataFrame to generate <code class="language-plaintext highlighter-rouge">wordCounts</code> is <em>exactly the same</em> 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 &#8220;incremental&#8221; query that combines the previous
running counts with the new data to compute updated counts, as shown below.</p>
<p><img src="../img/structured-streaming-example-model.png" alt="Model" /></p>
<p><strong>Note that Structured Streaming does not materialize the entire table</strong>. 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 <em>state</em> data as
required to update the result (e.g. intermediate counts in the earlier example).</p>
<p>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.</p>
<h2 id="handling-event-time-and-late-data">Handling Event-time and Late Data</h2>
<p>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 &#8211; 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 event-time column &#8211; 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.</p>
<p>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 <a href="./apis-on-dataframes-and-datasets.html#window-operations-on-event-time">Window Operations</a> section.</p>
<h2 id="fault-tolerance-semantics">Fault Tolerance Semantics</h2>
<p>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 <strong>end-to-end exactly-once semantics</strong> under any failure.</p>
</div>
<!-- /container -->
</div>
<script src="../js/vendor/jquery-3.5.1.min.js"></script>
<script src="../js/vendor/bootstrap.bundle.min.js"></script>
<script src="../js/vendor/anchor.min.js"></script>
<script src="../js/main.js"></script>
<script type="text/javascript" src="../js/vendor/docsearch.min.js"></script>
<script type="text/javascript">
// DocSearch is entirely free and automated. DocSearch is built in two parts:
// 1. a crawler which we run on our own infrastructure every 24 hours. It follows every link
// in your website and extract content from every page it traverses. It then pushes this
// content to an Algolia index.
// 2. a JavaScript snippet to be inserted in your website that will bind this Algolia index
// to your search input and display its results in a dropdown UI. If you want to find more
// details on how works DocSearch, check the docs of DocSearch.
docsearch({
apiKey: 'd62f962a82bc9abb53471cb7b89da35e',
appId: 'RAI69RXRSK',
indexName: 'apache_spark',
inputSelector: '#docsearch-input',
enhancedSearchInput: true,
algoliaOptions: {
'facetFilters': ["version:4.1.0-preview1"]
},
debug: false // Set debug to true if you want to inspect the dropdown
});
</script>
<!-- MathJax Section -->
<script type="text/x-mathjax-config">
MathJax.Hub.Config({
TeX: { equationNumbers: { autoNumber: "AMS" } }
});
</script>
<script>
// Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS.
// We could use "//cdn.mathjax...", but that won't support "file://".
(function(d, script) {
script = d.createElement('script');
script.type = 'text/javascript';
script.async = true;
script.onload = function(){
MathJax.Hub.Config({
tex2jax: {
inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
processEscapes: true,
skipTags: ['script', 'noscript', 'style', 'textarea', 'pre']
}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
'?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
</script>
</body>
</html>