| |
| <!DOCTYPE html> |
| <!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]--> |
| <!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]--> |
| <!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]--> |
| <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 3.5.0 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 type="text/javascript"> |
| 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"> |
| <!--[if lt IE 7]> |
| <p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p> |
| <![endif]--> |
| |
| <!-- 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="img/spark-logo-rev.svg" width="141" height="72"/></a><span class="version">3.5.0</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="structured-streaming-programming-guide.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> |
| </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/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/python/index.html">Python</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-mesos.html">Mesos</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">v3.5.0</span></span>--> |
| </div> |
| </nav> |
| |
| |
| |
| <div class="container"> |
| |
| |
| <div class="content mr-3" id="content"> |
| |
| |
| <h1 class="title">Structured Streaming Programming Guide</h1> |
| |
| |
| <ul id="markdown-toc"> |
| <li><a href="#overview" id="markdown-toc-overview">Overview</a></li> |
| <li><a href="#quick-example" id="markdown-toc-quick-example">Quick Example</a></li> |
| <li><a href="#programming-model" id="markdown-toc-programming-model">Programming Model</a> <ul> |
| <li><a href="#basic-concepts" id="markdown-toc-basic-concepts">Basic Concepts</a></li> |
| <li><a href="#handling-event-time-and-late-data" id="markdown-toc-handling-event-time-and-late-data">Handling Event-time and Late Data</a></li> |
| <li><a href="#fault-tolerance-semantics" id="markdown-toc-fault-tolerance-semantics">Fault Tolerance Semantics</a></li> |
| </ul> |
| </li> |
| <li><a href="#api-using-datasets-and-dataframes" id="markdown-toc-api-using-datasets-and-dataframes">API using Datasets and DataFrames</a> <ul> |
| <li><a href="#creating-streaming-dataframes-and-streaming-datasets" id="markdown-toc-creating-streaming-dataframes-and-streaming-datasets">Creating streaming DataFrames and streaming Datasets</a> <ul> |
| <li><a href="#input-sources" id="markdown-toc-input-sources">Input Sources</a></li> |
| <li><a href="#schema-inference-and-partition-of-streaming-dataframesdatasets" id="markdown-toc-schema-inference-and-partition-of-streaming-dataframesdatasets">Schema inference and partition of streaming DataFrames/Datasets</a></li> |
| </ul> |
| </li> |
| <li><a href="#operations-on-streaming-dataframesdatasets" id="markdown-toc-operations-on-streaming-dataframesdatasets">Operations on streaming DataFrames/Datasets</a> <ul> |
| <li><a href="#basic-operations---selection-projection-aggregation" id="markdown-toc-basic-operations---selection-projection-aggregation">Basic Operations - Selection, Projection, Aggregation</a></li> |
| <li><a href="#window-operations-on-event-time" id="markdown-toc-window-operations-on-event-time">Window Operations on Event Time</a> <ul> |
| <li><a href="#handling-late-data-and-watermarking" id="markdown-toc-handling-late-data-and-watermarking">Handling Late Data and Watermarking</a></li> |
| <li><a href="#types-of-time-windows" id="markdown-toc-types-of-time-windows">Types of time windows</a></li> |
| <li><a href="#representation-of-the-time-for-time-window" id="markdown-toc-representation-of-the-time-for-time-window">Representation of the time for time window</a></li> |
| </ul> |
| </li> |
| <li><a href="#join-operations" id="markdown-toc-join-operations">Join Operations</a> <ul> |
| <li><a href="#stream-static-joins" id="markdown-toc-stream-static-joins">Stream-static Joins</a></li> |
| <li><a href="#stream-stream-joins" id="markdown-toc-stream-stream-joins">Stream-stream Joins</a> <ul> |
| <li><a href="#inner-joins-with-optional-watermarking" id="markdown-toc-inner-joins-with-optional-watermarking">Inner Joins with optional Watermarking</a></li> |
| <li><a href="#outer-joins-with-watermarking" id="markdown-toc-outer-joins-with-watermarking">Outer Joins with Watermarking</a></li> |
| <li><a href="#semi-joins-with-watermarking" id="markdown-toc-semi-joins-with-watermarking">Semi Joins with Watermarking</a></li> |
| <li><a href="#support-matrix-for-joins-in-streaming-queries" id="markdown-toc-support-matrix-for-joins-in-streaming-queries">Support matrix for joins in streaming queries</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#streaming-deduplication" id="markdown-toc-streaming-deduplication">Streaming Deduplication</a></li> |
| <li><a href="#policy-for-handling-multiple-watermarks" id="markdown-toc-policy-for-handling-multiple-watermarks">Policy for handling multiple watermarks</a></li> |
| <li><a href="#arbitrary-stateful-operations" id="markdown-toc-arbitrary-stateful-operations">Arbitrary Stateful Operations</a></li> |
| <li><a href="#unsupported-operations" id="markdown-toc-unsupported-operations">Unsupported Operations</a></li> |
| <li><a href="#state-store" id="markdown-toc-state-store">State Store</a> <ul> |
| <li><a href="#hdfs-state-store-provider" id="markdown-toc-hdfs-state-store-provider">HDFS state store provider</a></li> |
| <li><a href="#rocksdb-state-store-implementation" id="markdown-toc-rocksdb-state-store-implementation">RocksDB state store implementation</a> <ul> |
| <li><a href="#rocksdb-state-store-memory-management" id="markdown-toc-rocksdb-state-store-memory-management">RocksDB State Store Memory Management</a></li> |
| <li><a href="#rocksdb-state-store-changelog-checkpointing" id="markdown-toc-rocksdb-state-store-changelog-checkpointing">RocksDB State Store Changelog Checkpointing</a></li> |
| <li><a href="#performance-aspect-considerations" id="markdown-toc-performance-aspect-considerations">Performance-aspect considerations</a></li> |
| </ul> |
| </li> |
| <li><a href="#state-store-and-task-locality" id="markdown-toc-state-store-and-task-locality">State Store and task locality</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#starting-streaming-queries" id="markdown-toc-starting-streaming-queries">Starting Streaming Queries</a> <ul> |
| <li><a href="#output-modes" id="markdown-toc-output-modes">Output Modes</a></li> |
| <li><a href="#output-sinks" id="markdown-toc-output-sinks">Output Sinks</a> <ul> |
| <li><a href="#using-foreach-and-foreachbatch" id="markdown-toc-using-foreach-and-foreachbatch">Using Foreach and ForeachBatch</a> <ul> |
| <li><a href="#foreachbatch" id="markdown-toc-foreachbatch">ForeachBatch</a></li> |
| <li><a href="#foreach" id="markdown-toc-foreach">Foreach</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#streaming-table-apis" id="markdown-toc-streaming-table-apis">Streaming Table APIs</a></li> |
| <li><a href="#triggers" id="markdown-toc-triggers">Triggers</a></li> |
| </ul> |
| </li> |
| <li><a href="#managing-streaming-queries" id="markdown-toc-managing-streaming-queries">Managing Streaming Queries</a></li> |
| <li><a href="#monitoring-streaming-queries" id="markdown-toc-monitoring-streaming-queries">Monitoring Streaming Queries</a> <ul> |
| <li><a href="#reading-metrics-interactively" id="markdown-toc-reading-metrics-interactively">Reading Metrics Interactively</a></li> |
| <li><a href="#reporting-metrics-programmatically-using-asynchronous-apis" id="markdown-toc-reporting-metrics-programmatically-using-asynchronous-apis">Reporting Metrics programmatically using Asynchronous APIs</a></li> |
| <li><a href="#reporting-metrics-using-dropwizard" id="markdown-toc-reporting-metrics-using-dropwizard">Reporting Metrics using Dropwizard</a></li> |
| </ul> |
| </li> |
| <li><a href="#recovering-from-failures-with-checkpointing" id="markdown-toc-recovering-from-failures-with-checkpointing">Recovering from Failures with Checkpointing</a></li> |
| <li><a href="#recovery-semantics-after-changes-in-a-streaming-query" id="markdown-toc-recovery-semantics-after-changes-in-a-streaming-query">Recovery Semantics after Changes in a Streaming Query</a></li> |
| </ul> |
| </li> |
| <li><a href="#asynchronous-progress-tracking" id="markdown-toc-asynchronous-progress-tracking">Asynchronous Progress Tracking</a> <ul> |
| <li><a href="#what-is-it" id="markdown-toc-what-is-it">What is it?</a></li> |
| <li><a href="#how-does-it-work" id="markdown-toc-how-does-it-work">How does it work?</a></li> |
| <li><a href="#how-to-use-it" id="markdown-toc-how-to-use-it">How to use it?</a></li> |
| <li><a href="#limitations" id="markdown-toc-limitations">Limitations</a></li> |
| <li><a href="#switching-the-setting-off" id="markdown-toc-switching-the-setting-off">Switching the setting off</a></li> |
| </ul> |
| </li> |
| <li><a href="#continuous-processing" id="markdown-toc-continuous-processing">Continuous Processing</a></li> |
| <li><a href="#additional-information" id="markdown-toc-additional-information">Additional Information</a></li> |
| <li><a href="#migration-guide" id="markdown-toc-migration-guide">Migration Guide</a></li> |
| </ul> |
| |
| <h1 id="overview">Overview</h1> |
| <p>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 <a href="sql-programming-guide.html">Dataset/DataFrame API</a> 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, <em>Structured Streaming provides fast, scalable, fault-tolerant, end-to-end exactly-once stream processing without the user having to reason about streaming.</em></p> |
| |
| <p>Internally, by default, Structured Streaming queries are processed using a <em>micro-batch processing</em> 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 <strong>Continuous Processing</strong>, 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.</p> |
| |
| <p>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 <a href="#continuous-processing">later</a> discuss Continuous Processing model. First, let’s start with a simple example of a Structured Streaming query - a streaming word count.</p> |
| |
| <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/v3.5.0/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala">Scala</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java">Java</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/python/sql/streaming/structured_network_wordcount.py">Python</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/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="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">explode</span> |
| <span class="kn">from</span> <span class="nn">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="n">appName</span><span class="p">(</span><span class="s">"StructuredNetworkWordCount"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">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="nb">format</span><span class="p">(</span><span class="s">"socket"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"host"</span><span class="p">,</span> <span class="s">"localhost"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"port"</span><span class="p">,</span> <span class="mi">9999</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">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="n">select</span><span class="p">(</span> |
| <span class="n">explode</span><span class="p">(</span> |
| <span class="n">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="s">" "</span><span class="p">)</span> |
| <span class="p">).</span><span class="n">alias</span><span class="p">(</span><span class="s">"word"</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="n">groupBy</span><span class="p">(</span><span class="s">"word"</span><span class="p">).</span><span class="n">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 “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function <code class="language-plaintext highlighter-rouge">alias</code> to name the new column as “word”. 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 “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 <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"><</span><span class="nc">Row</span><span class="o">></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"><</span><span class="nc">String</span><span class="o">></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"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">>)</span> <span class="n">x</span> <span class="o">-></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"><</span><span class="nc">Row</span><span class="o">></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 “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 <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"><-</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"><-</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"><-</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 “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 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 “word”. 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="n">outputMode</span><span class="p">(</span><span class="s">"complete"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="n">query</span><span class="p">.</span><span class="n">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"><-</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 “Input Table”. 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 “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.</p> |
| |
| <p><img src="img/structured-streaming-model.png" alt="Model" /></p> |
| |
| <p>The “Output” 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’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="#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 “incremental” 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 – 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 – 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="#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> |
| |
| <h1 id="api-using-datasets-and-dataframes">API using Datasets and DataFrames</h1> |
| <p>Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point <code class="language-plaintext highlighter-rouge">SparkSession</code> |
| (<a href="api/scala/org/apache/spark/sql/SparkSession.html">Scala</a>/<a href="api/java/org/apache/spark/sql/SparkSession.html">Java</a>/<a href="api/python/reference/pyspark.sql/api/pyspark.sql.SparkSession.html#pyspark.sql.SparkSession">Python</a>/<a href="api/R/reference/sparkR.session.html">R</a> docs) |
| to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the |
| <a href="sql-programming-guide.html">DataFrame/Dataset Programming Guide</a>.</p> |
| |
| <h2 id="creating-streaming-dataframes-and-streaming-datasets">Creating streaming DataFrames and streaming Datasets</h2> |
| <p>Streaming DataFrames can be created through the <code class="language-plaintext highlighter-rouge">DataStreamReader</code> interface |
| (<a href="api/scala/org/apache/spark/sql/streaming/DataStreamReader.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/DataStreamReader.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.DataStreamReader.html#pyspark.sql.streaming.DataStreamReader">Python</a> docs) |
| returned by <code class="language-plaintext highlighter-rouge">SparkSession.readStream()</code>. In <a href="api/R/reference/read.stream.html">R</a>, with the <code class="language-plaintext highlighter-rouge">read.stream()</code> method. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc.</p> |
| |
| <h4 id="input-sources">Input Sources</h4> |
| <p>There are a few built-in sources.</p> |
| |
| <ul> |
| <li><strong>File source</strong> - Reads files written in a directory as a stream of data. Files will be processed in the order of file modification time. If <code class="language-plaintext highlighter-rouge">latestFirst</code> 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.</li> |
| <li> |
| <p><strong>Kafka source</strong> - Reads data from Kafka. It’s compatible with Kafka broker versions 0.10.0 or higher. See the <a href="structured-streaming-kafka-integration.html">Kafka Integration Guide</a> for more details.</p> |
| </li> |
| <li> |
| <p><strong>Socket source (for testing)</strong> - 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.</p> |
| </li> |
| <li> |
| <p><strong>Rate source (for testing)</strong> - Generates data at the specified number of rows per second, each output row contains a <code class="language-plaintext highlighter-rouge">timestamp</code> and <code class="language-plaintext highlighter-rouge">value</code>. Where <code class="language-plaintext highlighter-rouge">timestamp</code> is a <code class="language-plaintext highlighter-rouge">Timestamp</code> type containing the time of message dispatch, and <code class="language-plaintext highlighter-rouge">value</code> is of <code class="language-plaintext highlighter-rouge">Long</code> type containing the message count, starting from 0 as the first row. This source is intended for testing and benchmarking.</p> |
| </li> |
| <li><strong>Rate Per Micro-Batch source (for testing)</strong> - Generates data at the specified number of rows per micro-batch, each output row contains a <code class="language-plaintext highlighter-rouge">timestamp</code> and <code class="language-plaintext highlighter-rouge">value</code>. Where <code class="language-plaintext highlighter-rouge">timestamp</code> is a <code class="language-plaintext highlighter-rouge">Timestamp</code> type containing the time of message dispatch, and <code class="language-plaintext highlighter-rouge">value</code> is of <code class="language-plaintext highlighter-rouge">Long</code> type containing the message count, starting from 0 as the first row. Unlike <code class="language-plaintext highlighter-rouge">rate</code> 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.</li> |
| </ul> |
| |
| <p>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 |
| <a href="#fault-tolerance-semantics">fault-tolerance semantics</a>. |
| Here are the details of all the sources in Spark.</p> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Source</th> |
| <th>Options</th> |
| <th>Fault-tolerant</th> |
| <th>Notes</th> |
| </tr> |
| </thead> |
| <tr> |
| <td><b>File source</b></td> |
| <td> |
| <code>path</code>: path to the input directory, and common to all file formats. |
| <br /> |
| <code>maxFilesPerTrigger</code>: maximum number of new files to be considered in every trigger (default: no max) |
| <br /> |
| <code>latestFirst</code>: whether to process the latest new files first, useful when there is a large backlog of files (default: false) |
| <br /> |
| <code>fileNameOnly</code>: whether to check new files based on only the filename instead of on the 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: |
| <br /> |
| "file:///dataset.txt"<br /> |
| "s3://a/dataset.txt"<br /> |
| "s3n://a/b/dataset.txt"<br /> |
| "s3a://a/b/c/dataset.txt" |
| <br /> |
| <code>maxFileAge</code>: 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 <code>latestFirst</code> is set to `true` and <code>maxFilesPerTrigger</code> 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) |
| <br /> |
| <code>cleanSource</code>: option to clean up completed files after processing.<br /> |
| Available options are "archive", "delete", "off". If the option is not provided, the default value is "off".<br /> |
| When "archive" is provided, additional option <code>sourceArchiveDir</code> 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.<br /> |
| 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.<br /> |
| Spark will move source files respecting their own path. For example, if the path of source file is <code>/a/b/dataset.txt</code> and the path of archive directory is <code>/archived/here</code>, file will be moved to <code>/archived/here/a/b/dataset.txt</code>.<br /> |
| 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.<br /> |
| Number of threads used in completed file cleaner can be configured with <code>spark.sql.streaming.fileSource.cleaner.numThreads</code> (default: 1).<br /> |
| 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.<br /> |
| 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. |
| <br /><br /> |
| For file-format-specific options, see the related methods in <code>DataStreamReader</code> |
| (<a href="api/scala/org/apache/spark/sql/streaming/DataStreamReader.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/DataStreamReader.html">Java</a>/<a href="api/python/reference/pyspark.sql/api/pyspark.sql.streaming.DataStreamReader.html#pyspark.sql.streaming.DataStreamReader">Python</a>/<a href="api/R/read.stream.html">R</a>). |
| E.g. for "parquet" format options see <code>DataStreamReader.parquet()</code>. |
| <br /><br /> |
| In addition, there are session configurations that affect certain file-formats. See the <a href="sql-programming-guide.html">SQL Programming Guide</a> for more details. E.g., for "parquet", see <a href="sql-data-sources-parquet.html#configuration">Parquet configuration</a> section. |
| </td> |
| <td>Yes</td> |
| <td>Supports glob paths, but does not support multiple comma-separated paths/globs.</td> |
| </tr> |
| <tr> |
| <td><b>Socket Source</b></td> |
| <td> |
| <code>host</code>: host to connect to, must be specified<br /> |
| <code>port</code>: port to connect to, must be specified |
| </td> |
| <td>No</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td><b>Rate Source</b></td> |
| <td> |
| <code>rowsPerSecond</code> (e.g. 100, default: 1): How many rows should be generated per second.<br /><br /> |
| <code>rampUpTime</code> (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes <code>rowsPerSecond</code>. Using finer granularities than seconds will be truncated to integer seconds. <br /><br /> |
| <code>numPartitions</code> (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. <br /><br /> |
| |
| The source will try its best to reach <code>rowsPerSecond</code>, but the query may be resource constrained, and <code>numPartitions</code> can be tweaked to help reach the desired speed. |
| </td> |
| <td>Yes</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td><b>Rate Per Micro-Batch Source</b> (format: <b>rate-micro-batch</b>)</td> |
| <td> |
| <code>rowsPerBatch</code> (e.g. 100): How many rows should be generated per micro-batch.<br /><br /> |
| <code>numPartitions</code> (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. <br /><br /> |
| <code>startTimestamp</code> (e.g. 1000, default: 0): starting value of generated time. <br /><br /> |
| <code>advanceMillisPerBatch</code> (e.g. 1000, default: 1000): the amount of time being advanced in generated time on each micro-batch. <br /><br /> |
| </td> |
| <td>Yes</td> |
| <td></td> |
| </tr> |
| |
| <tr> |
| <td><b>Kafka Source</b></td> |
| <td> |
| See the <a href="structured-streaming-kafka-integration.html">Kafka Integration Guide</a>. |
| </td> |
| <td>Yes</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td></td> |
| <td></td> |
| <td></td> |
| <td></td> |
| </tr> |
| </table> |
| |
| <p>Here are some examples.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span><span class="p">.</span> <span class="p">...</span> |
| |
| <span class="c1"># Read text from socket |
| </span><span class="n">socketDF</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="nb">format</span><span class="p">(</span><span class="s">"socket"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"host"</span><span class="p">,</span> <span class="s">"localhost"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"port"</span><span class="p">,</span> <span class="mi">9999</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">load</span><span class="p">()</span> |
| |
| <span class="n">socketDF</span><span class="p">.</span><span class="n">isStreaming</span><span class="p">()</span> <span class="c1"># Returns True for DataFrames that have streaming sources |
| </span> |
| <span class="n">socketDF</span><span class="p">.</span><span class="n">printSchema</span><span class="p">()</span> |
| |
| <span class="c1"># Read all the csv files written atomically in a directory |
| </span><span class="n">userSchema</span> <span class="o">=</span> <span class="n">StructType</span><span class="p">().</span><span class="n">add</span><span class="p">(</span><span class="s">"name"</span><span class="p">,</span> <span class="s">"string"</span><span class="p">).</span><span class="n">add</span><span class="p">(</span><span class="s">"age"</span><span class="p">,</span> <span class="s">"integer"</span><span class="p">)</span> |
| <span class="n">csvDF</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="n">option</span><span class="p">(</span><span class="s">"sep"</span><span class="p">,</span> <span class="s">";"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">schema</span><span class="p">(</span><span class="n">userSchema</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">csv</span><span class="p">(</span><span class="s">"/path/to/directory"</span><span class="p">)</span> <span class="c1"># Equivalent to format("csv").load("/path/to/directory")</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">spark</span><span class="k">:</span> <span class="kt">SparkSession</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="c1">// Read text from socket</span> |
| <span class="k">val</span> <span class="nv">socketDF</span> <span class="k">=</span> <span class="n">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="nv">socketDF</span><span class="o">.</span><span class="py">isStreaming</span> <span class="c1">// Returns True for DataFrames that have streaming sources</span> |
| |
| <span class="nv">socketDF</span><span class="o">.</span><span class="py">printSchema</span> |
| |
| <span class="c1">// Read all the csv files written atomically in a directory</span> |
| <span class="k">val</span> <span class="nv">userSchema</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">StructType</span><span class="o">().</span><span class="py">add</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"string"</span><span class="o">).</span><span class="py">add</span><span class="o">(</span><span class="s">"age"</span><span class="o">,</span> <span class="s">"integer"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="nv">csvDF</span> <span class="k">=</span> <span class="n">spark</span> |
| <span class="o">.</span><span class="py">readStream</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"sep"</span><span class="o">,</span> <span class="s">";"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">schema</span><span class="o">(</span><span class="n">userSchema</span><span class="o">)</span> <span class="c1">// Specify schema of the csv files</span> |
| <span class="o">.</span><span class="py">csv</span><span class="o">(</span><span class="s">"/path/to/directory"</span><span class="o">)</span> <span class="c1">// Equivalent to format("csv").load("/path/to/directory")</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="nc">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="c1">// Read text from socket</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">socketDF</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="n">socketDF</span><span class="o">.</span><span class="na">isStreaming</span><span class="o">();</span> <span class="c1">// Returns True for DataFrames that have streaming sources</span> |
| |
| <span class="n">socketDF</span><span class="o">.</span><span class="na">printSchema</span><span class="o">();</span> |
| |
| <span class="c1">// Read all the csv files written atomically in a directory</span> |
| <span class="nc">StructType</span> <span class="n">userSchema</span> <span class="o">=</span> <span class="k">new</span> <span class="nc">StructType</span><span class="o">().</span><span class="na">add</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"string"</span><span class="o">).</span><span class="na">add</span><span class="o">(</span><span class="s">"age"</span><span class="o">,</span> <span class="s">"integer"</span><span class="o">);</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">csvDF</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">option</span><span class="o">(</span><span class="s">"sep"</span><span class="o">,</span> <span class="s">";"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">schema</span><span class="o">(</span><span class="n">userSchema</span><span class="o">)</span> <span class="c1">// Specify schema of the csv files</span> |
| <span class="o">.</span><span class="na">csv</span><span class="o">(</span><span class="s">"/path/to/directory"</span><span class="o">);</span> <span class="c1">// Equivalent to format("csv").load("/path/to/directory")</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">...</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Read text from socket</span><span class="w"> |
| </span><span class="n">socketDF</span><span class="w"> </span><span class="o"><-</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="n">hostname</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="n">port</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="n">isStreaming</span><span class="p">(</span><span class="n">socketDF</span><span class="p">)</span><span class="w"> </span><span class="c1"># Returns TRUE for SparkDataFrames that have streaming sources</span><span class="w"> |
| |
| </span><span class="n">printSchema</span><span class="p">(</span><span class="n">socketDF</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Read all the csv files written atomically in a directory</span><span class="w"> |
| </span><span class="n">schema</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">structType</span><span class="p">(</span><span class="n">structField</span><span class="p">(</span><span class="s2">"name"</span><span class="p">,</span><span class="w"> </span><span class="s2">"string"</span><span class="p">),</span><span class="w"> |
| </span><span class="n">structField</span><span class="p">(</span><span class="s2">"age"</span><span class="p">,</span><span class="w"> </span><span class="s2">"integer"</span><span class="p">))</span><span class="w"> |
| </span><span class="n">csvDF</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="s2">"csv"</span><span class="p">,</span><span class="w"> </span><span class="n">path</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"/path/to/directory"</span><span class="p">,</span><span class="w"> </span><span class="n">schema</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">schema</span><span class="p">,</span><span class="w"> </span><span class="n">sep</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">";"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <p>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 <code class="language-plaintext highlighter-rouge">map</code>, <code class="language-plaintext highlighter-rouge">flatMap</code>, 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 <a href="sql-programming-guide.html">SQL Programming Guide</a> for more details. Additionally, more details on the supported streaming sources are discussed later in the document.</p> |
| |
| <p>Since Spark 3.1, you can also create streaming DataFrames from tables with <code class="language-plaintext highlighter-rouge">DataStreamReader.table()</code>. See <a href="#streaming-table-apis">Streaming Table APIs</a> for more details.</p> |
| |
| <h3 id="schema-inference-and-partition-of-streaming-dataframesdatasets">Schema inference and partition of streaming DataFrames/Datasets</h3> |
| |
| <p>By default, Structured Streaming from file based sources requires you to specify the schema, rather than rely on Spark to infer it automatically. This restriction ensures a consistent schema will be used for the streaming query, even in the case of failures. For ad-hoc use cases, you can reenable schema inference by setting <code class="language-plaintext highlighter-rouge">spark.sql.streaming.schemaInference</code> to <code class="language-plaintext highlighter-rouge">true</code>.</p> |
| |
| <p>Partition discovery does occur when subdirectories that are named <code class="language-plaintext highlighter-rouge">/key=value/</code> 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 <code class="language-plaintext highlighter-rouge">/data/year=2016/</code> when <code class="language-plaintext highlighter-rouge">/data/year=2015/</code> was present, but it is invalid to change the partitioning column (i.e. by creating the directory <code class="language-plaintext highlighter-rouge">/data/date=2016-04-17/</code>).</p> |
| |
| <h2 id="operations-on-streaming-dataframesdatasets">Operations on streaming DataFrames/Datasets</h2> |
| <p>You can apply all kinds of operations on streaming DataFrames/Datasets – ranging from untyped, SQL-like operations (e.g. <code class="language-plaintext highlighter-rouge">select</code>, <code class="language-plaintext highlighter-rouge">where</code>, <code class="language-plaintext highlighter-rouge">groupBy</code>), to typed RDD-like operations (e.g. <code class="language-plaintext highlighter-rouge">map</code>, <code class="language-plaintext highlighter-rouge">filter</code>, <code class="language-plaintext highlighter-rouge">flatMap</code>). See the <a href="sql-programming-guide.html">SQL programming guide</a> for more details. Let’s take a look at a few example operations that you can use.</p> |
| |
| <h3 id="basic-operations---selection-projection-aggregation">Basic Operations - Selection, Projection, Aggregation</h3> |
| <p>Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are <a href="#unsupported-operations">discussed later</a> in this section.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">df</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: DateType } |
| </span> |
| <span class="c1"># Select the devices which have signal more than 10 |
| </span><span class="n">df</span><span class="p">.</span><span class="n">select</span><span class="p">(</span><span class="s">"device"</span><span class="p">).</span><span class="n">where</span><span class="p">(</span><span class="s">"signal > 10"</span><span class="p">)</span> |
| |
| <span class="c1"># Running count of the number of updates for each device type |
| </span><span class="n">df</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s">"deviceType"</span><span class="p">).</span><span class="n">count</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">case</span> <span class="k">class</span> <span class="nc">DeviceData</span><span class="o">(</span><span class="n">device</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="n">deviceType</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="n">signal</span><span class="k">:</span> <span class="kt">Double</span><span class="o">,</span> <span class="n">time</span><span class="k">:</span> <span class="kt">DateTime</span><span class="o">)</span> |
| |
| <span class="k">val</span> <span class="nv">df</span><span class="k">:</span> <span class="kt">DataFrame</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: string }</span> |
| <span class="k">val</span> <span class="nv">ds</span><span class="k">:</span> <span class="kt">Dataset</span><span class="o">[</span><span class="kt">DeviceData</span><span class="o">]</span> <span class="k">=</span> <span class="nv">df</span><span class="o">.</span><span class="py">as</span><span class="o">[</span><span class="kt">DeviceData</span><span class="o">]</span> <span class="c1">// streaming Dataset with IOT device data</span> |
| |
| <span class="c1">// Select the devices which have signal more than 10</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">select</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="py">where</span><span class="o">(</span><span class="s">"signal > 10"</span><span class="o">)</span> <span class="c1">// using untyped APIs</span> |
| <span class="nv">ds</span><span class="o">.</span><span class="py">filter</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">signal</span> <span class="o">></span> <span class="mi">10</span><span class="o">).</span><span class="py">map</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">device</span><span class="o">)</span> <span class="c1">// using typed APIs</span> |
| |
| <span class="c1">// Running count of the number of updates for each device type</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="s">"deviceType"</span><span class="o">).</span><span class="py">count</span><span class="o">()</span> <span class="c1">// using untyped API</span> |
| |
| <span class="c1">// Running average signal for each device type</span> |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.expressions.scalalang.typed</span> |
| <span class="nv">ds</span><span class="o">.</span><span class="py">groupByKey</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">deviceType</span><span class="o">).</span><span class="py">agg</span><span class="o">(</span><span class="nv">typed</span><span class="o">.</span><span class="py">avg</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">signal</span><span class="o">))</span> <span class="c1">// using typed API</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.*</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.expressions.javalang.typed</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.catalyst.encoders.ExpressionEncoder</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">class</span> <span class="nc">DeviceData</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="nc">String</span> <span class="n">device</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="nc">String</span> <span class="n">deviceType</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="nc">Double</span> <span class="n">signal</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="n">java</span><span class="o">.</span><span class="na">sql</span><span class="o">.</span><span class="na">Date</span> <span class="n">time</span><span class="o">;</span> |
| <span class="o">...</span> |
| <span class="c1">// Getter and setter methods for each field</span> |
| <span class="o">}</span> |
| |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">df</span> <span class="o">=</span> <span class="o">...;</span> <span class="c1">// streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType }</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">DeviceData</span><span class="o">></span> <span class="n">ds</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="nc">ExpressionEncoder</span><span class="o">.</span><span class="na">javaBean</span><span class="o">(</span><span class="nc">DeviceData</span><span class="o">.</span><span class="na">class</span><span class="o">));</span> <span class="c1">// streaming Dataset with IOT device data</span> |
| |
| <span class="c1">// Select the devices which have signal more than 10</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="na">where</span><span class="o">(</span><span class="s">"signal > 10"</span><span class="o">);</span> <span class="c1">// using untyped APIs</span> |
| <span class="n">ds</span><span class="o">.</span><span class="na">filter</span><span class="o">((</span><span class="nc">FilterFunction</span><span class="o"><</span><span class="nc">DeviceData</span><span class="o">>)</span> <span class="n">value</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">getSignal</span><span class="o">()</span> <span class="o">></span> <span class="mi">10</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">map</span><span class="o">((</span><span class="nc">MapFunction</span><span class="o"><</span><span class="nc">DeviceData</span><span class="o">,</span> <span class="nc">String</span><span class="o">>)</span> <span class="n">value</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">getDevice</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">// Running count of the number of updates for each device type</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"deviceType"</span><span class="o">).</span><span class="na">count</span><span class="o">();</span> <span class="c1">// using untyped API</span> |
| |
| <span class="c1">// Running average signal for each device type</span> |
| <span class="n">ds</span><span class="o">.</span><span class="na">groupByKey</span><span class="o">((</span><span class="nc">MapFunction</span><span class="o"><</span><span class="nc">DeviceData</span><span class="o">,</span> <span class="nc">String</span><span class="o">>)</span> <span class="n">value</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">getDeviceType</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">agg</span><span class="o">(</span><span class="n">typed</span><span class="o">.</span><span class="na">avg</span><span class="o">((</span><span class="nc">MapFunction</span><span class="o"><</span><span class="nc">DeviceData</span><span class="o">,</span> <span class="nc">Double</span><span class="o">>)</span> <span class="n">value</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">getSignal</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">df</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">...</span><span class="w"> </span><span class="c1"># streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: DateType }</span><span class="w"> |
| |
| </span><span class="c1"># Select the devices which have signal more than 10</span><span class="w"> |
| </span><span class="n">select</span><span class="p">(</span><span class="n">where</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"signal > 10"</span><span class="p">),</span><span class="w"> </span><span class="s2">"device"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Running count of the number of updates for each device type</span><span class="w"> |
| </span><span class="n">count</span><span class="p">(</span><span class="n">groupBy</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"deviceType"</span><span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>You can also register a streaming DataFrame/Dataset as a temporary view and then apply SQL commands on it.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">df</span><span class="p">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s">"updates"</span><span class="p">)</span> |
| <span class="n">spark</span><span class="p">.</span><span class="n">sql</span><span class="p">(</span><span class="s">"select count(*) from updates"</span><span class="p">)</span> <span class="c1"># returns another streaming DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="nv">df</span><span class="o">.</span><span class="py">createOrReplaceTempView</span><span class="o">(</span><span class="s">"updates"</span><span class="o">)</span> |
| <span class="nv">spark</span><span class="o">.</span><span class="py">sql</span><span class="o">(</span><span class="s">"select count(*) from updates"</span><span class="o">)</span> <span class="c1">// returns another streaming DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">df</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"updates"</span><span class="o">);</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"select count(*) from updates"</span><span class="o">);</span> <span class="c1">// returns another streaming DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"updates"</span><span class="p">)</span><span class="w"> |
| </span><span class="n">sql</span><span class="p">(</span><span class="s2">"select count(*) from updates"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>Note, you can identify whether a DataFrame/Dataset has streaming data or not by using <code class="language-plaintext highlighter-rouge">df.isStreaming</code>.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">df</span><span class="p">.</span><span class="n">isStreaming</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="nv">df</span><span class="o">.</span><span class="py">isStreaming</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">df</span><span class="o">.</span><span class="na">isStreaming</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">isStreaming</span><span class="p">(</span><span class="n">df</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>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.)</p> |
| |
| <h3 id="window-operations-on-event-time">Window Operations on Event Time</h3> |
| <p>Aggregations over a sliding event-time window are straightforward with Structured Streaming and are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let’s understand this with an illustration.</p> |
| |
| <p>Imagine our <a href="#quick-example">quick example</a> 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).</p> |
| |
| <p>The result tables would look something like the following.</p> |
| |
| <p><img src="img/structured-streaming-window.png" alt="Window Operations" /></p> |
| |
| <p>Since this windowing is similar to grouping, in code, you can use <code class="language-plaintext highlighter-rouge">groupBy()</code> and <code class="language-plaintext highlighter-rouge">window()</code> operations to express windowed aggregations. You can see the full code for the below examples in |
| <a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala">Scala</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java">Java</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py">Python</a>.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">words</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String } |
| </span> |
| <span class="c1"># Group the data by window and word and compute the count of each group |
| </span><span class="n">windowedCounts</span> <span class="o">=</span> <span class="n">words</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">,</span> <span class="s">"5 minutes"</span><span class="p">),</span> |
| <span class="n">words</span><span class="p">.</span><span class="n">word</span> |
| <span class="p">).</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">words</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">windowedCounts</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="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span> |
| <span class="o">).</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">windowedCounts</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="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">).</span><span class="na">count</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">words</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">...</span><span class="w"> </span><span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String }</span><span class="w"> |
| |
| </span><span class="c1"># Group the data by window and word and compute the count of each group</span><span class="w"> |
| </span><span class="n">windowedCounts</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">count</span><span class="p">(</span><span class="w"> |
| </span><span class="n">groupBy</span><span class="p">(</span><span class="w"> |
| </span><span class="n">words</span><span class="p">,</span><span class="w"> |
| </span><span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="o">$</span><span class="n">timestamp</span><span class="p">,</span><span class="w"> </span><span class="s2">"10 minutes"</span><span class="p">,</span><span class="w"> </span><span class="s2">"5 minutes"</span><span class="p">),</span><span class="w"> |
| </span><span class="n">words</span><span class="o">$</span><span class="n">word</span><span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h4 id="handling-late-data-and-watermarking">Handling Late Data and Watermarking</h4> |
| <p>Now consider what happens if one of the events arrives late to the application. |
| For example, say, a word generated at 12:04 (i.e. event time) could be received by |
| the application at 12:11. The application should use the time 12:04 instead of 12:11 |
| to update the older counts for the window <code class="language-plaintext highlighter-rouge">12:00 - 12:10</code>. This occurs |
| naturally in our window-based grouping – Structured Streaming can maintain the intermediate state |
| for partial aggregates for a long period of time such that late data can update aggregates of |
| old windows correctly, as illustrated below.</p> |
| |
| <p><img src="img/structured-streaming-late-data.png" alt="Handling Late Data" /></p> |
| |
| <p>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 |
| <strong>watermarking</strong>, 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 <code class="language-plaintext highlighter-rouge">T</code>, the engine will maintain state and allow late |
| data to update the state until <code class="language-plaintext highlighter-rouge">(max event time seen by the engine - late threshold > T)</code>. |
| In other words, late data within the threshold will be aggregated, |
| but data later than the threshold will start getting dropped |
| (see <a href="#semantic-guarantees-of-aggregation-with-watermarking">later</a> |
| in the section for the exact guarantees). Let’s understand this with an example. We can |
| easily define watermarking on the previous example using <code class="language-plaintext highlighter-rouge">withWatermark()</code> as shown below.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">words</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String } |
| </span> |
| <span class="c1"># Group the data by window and word and compute the count of each group |
| </span><span class="n">windowedCounts</span> <span class="o">=</span> <span class="n">words</span> \ |
| <span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"timestamp"</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">,</span> <span class="s">"5 minutes"</span><span class="p">),</span> |
| <span class="n">words</span><span class="p">.</span><span class="n">word</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">words</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">windowedCounts</span> <span class="k">=</span> <span class="n">words</span> |
| <span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span> |
| <span class="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">windowedCounts</span> <span class="o">=</span> <span class="n">words</span> |
| <span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
| <span class="n">window</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</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">words</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">...</span><span class="w"> </span><span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String }</span><span class="w"> |
| |
| </span><span class="c1"># Group the data by window and word and compute the count of each group</span><span class="w"> |
| |
| </span><span class="n">words</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">withWatermark</span><span class="p">(</span><span class="n">words</span><span class="p">,</span><span class="w"> </span><span class="s2">"timestamp"</span><span class="p">,</span><span class="w"> </span><span class="s2">"10 minutes"</span><span class="p">)</span><span class="w"> |
| </span><span class="n">windowedCounts</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">count</span><span class="p">(</span><span class="w"> |
| </span><span class="n">groupBy</span><span class="p">(</span><span class="w"> |
| </span><span class="n">words</span><span class="p">,</span><span class="w"> |
| </span><span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="o">$</span><span class="n">timestamp</span><span class="p">,</span><span class="w"> </span><span class="s2">"10 minutes"</span><span class="p">,</span><span class="w"> </span><span class="s2">"5 minutes"</span><span class="p">),</span><span class="w"> |
| </span><span class="n">words</span><span class="o">$</span><span class="n">word</span><span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>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 <a href="#output-modes">Output Modes</a> 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.</p> |
| |
| <p><img src="img/structured-streaming-watermark-update-mode.png" alt="Watermarking in Update Mode" /></p> |
| |
| <p>As shown in the illustration, the maximum event time tracked by the engine is the |
| <em>blue dashed line</em>, and the watermark set as <code class="language-plaintext highlighter-rouge">(max event time - '10 mins')</code> |
| at the beginning of every trigger is the red line. For example, when the engine observes the data |
| <code class="language-plaintext highlighter-rouge">(12:14, dog)</code>, it sets the watermark for the next trigger as <code class="language-plaintext highlighter-rouge">12:04</code>. |
| This watermark lets the engine maintain intermediate state for additional 10 minutes to allow late |
| data to be counted. For example, the data <code class="language-plaintext highlighter-rouge">(12:09, cat)</code> is out of order and late, and it falls in |
| windows <code class="language-plaintext highlighter-rouge">12:00 - 12:10</code> and <code class="language-plaintext highlighter-rouge">12:05 - 12:15</code>. Since, it is still ahead of the watermark <code class="language-plaintext highlighter-rouge">12:04</code> 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 <code class="language-plaintext highlighter-rouge">12:11</code>, the intermediate |
| state for window <code class="language-plaintext highlighter-rouge">(12:00 - 12:10)</code> is cleared, and all subsequent data (e.g. <code class="language-plaintext highlighter-rouge">(12:04, donkey)</code>) |
| 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.</p> |
| |
| <p>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 <em>final counts</em> are written to sink. |
| This is illustrated below.</p> |
| |
| <p>Note that using <code class="language-plaintext highlighter-rouge">withWatermark</code> 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.</p> |
| |
| <p><img src="img/structured-streaming-watermark-append-mode.png" alt="Watermarking in Append Mode" /></p> |
| |
| <p>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 <code class="language-plaintext highlighter-rouge">12:00 - 12:10</code> is |
| appended to the Result Table only after the watermark is updated to <code class="language-plaintext highlighter-rouge">12:11</code>.</p> |
| |
| <h4 id="types-of-time-windows">Types of time windows</h4> |
| |
| <p>Spark supports three types of time windows: tumbling (fixed), sliding and session.</p> |
| |
| <p><img src="img/structured-streaming-time-window-types.jpg" alt="The types of time windows" /></p> |
| |
| <p>Tumbling windows are a series of fixed-sized, non-overlapping and contiguous time intervals. An input |
| can only be bound to a single window.</p> |
| |
| <p>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.</p> |
| |
| <p>Tumbling and sliding window use <code class="language-plaintext highlighter-rouge">window</code> function, which has been described on above examples.</p> |
| |
| <p>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.</p> |
| |
| <p>Session window uses <code class="language-plaintext highlighter-rouge">session_window</code> function. The usage of the function is similar to the <code class="language-plaintext highlighter-rouge">window</code> function.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">events</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, userId: String } |
| </span> |
| <span class="c1"># Group the data by session window and userId, and compute the count of each group |
| </span><span class="n">sessionizedCounts</span> <span class="o">=</span> <span class="n">events</span> \ |
| <span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"timestamp"</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">session_window</span><span class="p">(</span><span class="n">events</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="s">"5 minutes"</span><span class="p">),</span> |
| <span class="n">events</span><span class="p">.</span><span class="n">userId</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">events</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, userId: String }</span> |
| |
| <span class="c1">// Group the data by session window and userId, and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">sessionizedCounts</span> <span class="k">=</span> <span class="n">events</span> |
| <span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span> |
| <span class="nf">session_window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"userId"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">events</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, userId: String }</span> |
| |
| <span class="c1">// Group the data by session window and userId, and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">sessionizedCounts</span> <span class="o">=</span> <span class="n">events</span> |
| <span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
| <span class="n">session_window</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">col</span><span class="o">(</span><span class="s">"userId"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>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.</p> |
| |
| <p>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.</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="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">functions</span> <span class="k">as</span> <span class="n">sf</span> |
| |
| <span class="n">events</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, userId: String } |
| </span> |
| <span class="n">session_window</span> <span class="o">=</span> <span class="n">session_window</span><span class="p">(</span><span class="n">events</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> \ |
| <span class="n">sf</span><span class="p">.</span><span class="n">when</span><span class="p">(</span><span class="n">events</span><span class="p">.</span><span class="n">userId</span> <span class="o">==</span> <span class="s">"user1"</span><span class="p">,</span> <span class="s">"5 seconds"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">when</span><span class="p">(</span><span class="n">events</span><span class="p">.</span><span class="n">userId</span> <span class="o">==</span> <span class="s">"user2"</span><span class="p">,</span> <span class="s">"20 seconds"</span><span class="p">).</span><span class="n">otherwise</span><span class="p">(</span><span class="s">"5 minutes"</span><span class="p">))</span> |
| |
| <span class="c1"># Group the data by session window and userId, and compute the count of each group |
| </span><span class="n">sessionizedCounts</span> <span class="o">=</span> <span class="n">events</span> \ |
| <span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"timestamp"</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">session_window</span><span class="p">,</span> |
| <span class="n">events</span><span class="p">.</span><span class="n">userId</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">events</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, userId: String }</span> |
| |
| <span class="k">val</span> <span class="nv">sessionWindow</span> <span class="k">=</span> <span class="nf">session_window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="nf">when</span><span class="o">(</span><span class="n">$</span><span class="s">"userId"</span> <span class="o">===</span> <span class="s">"user1"</span><span class="o">,</span> <span class="s">"5 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">when</span><span class="o">(</span><span class="n">$</span><span class="s">"userId"</span> <span class="o">===</span> <span class="s">"user2"</span><span class="o">,</span> <span class="s">"20 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">otherwise</span><span class="o">(</span><span class="s">"5 minutes"</span><span class="o">))</span> |
| |
| <span class="c1">// Group the data by session window and userId, and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">sessionizedCounts</span> <span class="k">=</span> <span class="n">events</span> |
| <span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span> |
| <span class="nc">Column</span><span class="o">(</span><span class="n">sessionWindow</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"userId"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">events</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, userId: String }</span> |
| |
| <span class="nc">SessionWindow</span> <span class="n">sessionWindow</span> <span class="o">=</span> <span class="n">session_window</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="n">when</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"userId"</span><span class="o">).</span><span class="na">equalTo</span><span class="o">(</span><span class="s">"user1"</span><span class="o">),</span> <span class="s">"5 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">when</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"userId"</span><span class="o">).</span><span class="na">equalTo</span><span class="o">(</span><span class="s">"user2"</span><span class="o">),</span> <span class="s">"20 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">otherwise</span><span class="o">(</span><span class="s">"5 minutes"</span><span class="o">))</span> |
| |
| <span class="c1">// Group the data by session window and userId, and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">sessionizedCounts</span> <span class="o">=</span> <span class="n">events</span> |
| <span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
| <span class="k">new</span> <span class="nf">Column</span><span class="o">(</span><span class="n">sessionWindow</span><span class="o">),</span> |
| <span class="n">col</span><span class="o">(</span><span class="s">"userId"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>Note that there are some restrictions when you use session window in streaming query, like below:</p> |
| |
| <ul> |
| <li>“Update mode” as output mode is not supported.</li> |
| <li>There should be at least one column in addition to <code class="language-plaintext highlighter-rouge">session_window</code> in grouping key.</li> |
| </ul> |
| |
| <p>For batch query, global window (only having <code class="language-plaintext highlighter-rouge">session_window</code> in grouping key) is supported.</p> |
| |
| <p>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.</p> |
| |
| <p>You can enable <code class="language-plaintext highlighter-rouge">spark.sql.streaming.sessionWindow.merge.sessions.in.local.partition</code> to indicate Spark to perform partial aggregation.</p> |
| |
| <h4 id="representation-of-the-time-for-time-window">Representation of the time for time window</h4> |
| |
| <p>In some use cases, it is necessary to extract the representation of the time for time window, to apply operations requiring timestamp to the time windowed data. |
| One example is chained time window aggregations, where users want to define another time window against the time window. Say, someone wants to aggregate 5 minutes time windows as 1 hour tumble time window.</p> |
| |
| <p>There are two ways to achieve this, like below:</p> |
| |
| <ol> |
| <li>Use <code class="language-plaintext highlighter-rouge">window_time</code> SQL function with time window column as parameter</li> |
| <li>Use <code class="language-plaintext highlighter-rouge">window</code> SQL function with time window column as parameter</li> |
| </ol> |
| |
| <p><code class="language-plaintext highlighter-rouge">window_time</code> function will produce a timestamp which represents the time for time window. |
| User can pass the result to the parameter of <code class="language-plaintext highlighter-rouge">window</code> function (or anywhere requiring timestamp) to perform operation(s) with time window which requires timestamp.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">words</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String } |
| </span> |
| <span class="c1"># Group the data by window and word and compute the count of each group |
| </span><span class="n">windowedCounts</span> <span class="o">=</span> <span class="n">words</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">,</span> <span class="s">"5 minutes"</span><span class="p">),</span> |
| <span class="n">words</span><span class="p">.</span><span class="n">word</span> |
| <span class="p">).</span><span class="n">count</span><span class="p">()</span> |
| |
| <span class="c1"># Group the windowed data by another window and word and compute the count of each group |
| </span><span class="n">anotherWindowedCounts</span> <span class="o">=</span> <span class="n">windowedCounts</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">window_time</span><span class="p">(</span><span class="n">windowedCounts</span><span class="p">.</span><span class="n">window</span><span class="p">),</span> <span class="s">"1 hour"</span><span class="p">),</span> |
| <span class="n">windowedCounts</span><span class="p">.</span><span class="n">word</span> |
| <span class="p">).</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">words</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">windowedCounts</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="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span> |
| <span class="o">).</span><span class="py">count</span><span class="o">()</span> |
| |
| <span class="c1">// Group the windowed data by another window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">anotherWindowedCounts</span> <span class="k">=</span> <span class="nv">windowedCounts</span><span class="o">.</span><span class="py">groupBy</span><span class="o">(</span> |
| <span class="nf">window</span><span class="o">(</span><span class="nf">window_time</span><span class="o">(</span><span class="n">$</span><span class="s">"window"</span><span class="o">),</span> <span class="s">"1 hour"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span> |
| <span class="o">).</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">windowedCounts</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="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">).</span><span class="na">count</span><span class="o">();</span> |
| |
| <span class="c1">// Group the windowed data by another window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">anotherWindowedCounts</span> <span class="o">=</span> <span class="n">windowedCounts</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
| <span class="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">functions</span><span class="o">.</span><span class="na">window_time</span><span class="o">(</span><span class="s">"window"</span><span class="o">),</span> <span class="s">"1 hour"</span><span class="o">),</span> |
| <span class="n">windowedCounts</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">).</span><span class="na">count</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p><code class="language-plaintext highlighter-rouge">window</code> 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.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">words</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># streaming DataFrame of schema { timestamp: Timestamp, word: String } |
| </span> |
| <span class="c1"># Group the data by window and word and compute the count of each group |
| </span><span class="n">windowedCounts</span> <span class="o">=</span> <span class="n">words</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">words</span><span class="p">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="s">"10 minutes"</span><span class="p">,</span> <span class="s">"5 minutes"</span><span class="p">),</span> |
| <span class="n">words</span><span class="p">.</span><span class="n">word</span> |
| <span class="p">).</span><span class="n">count</span><span class="p">()</span> |
| |
| <span class="c1"># Group the windowed data by another window and word and compute the count of each group |
| </span><span class="n">anotherWindowedCounts</span> <span class="o">=</span> <span class="n">windowedCounts</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">windowedCounts</span><span class="p">.</span><span class="n">window</span><span class="p">,</span> <span class="s">"1 hour"</span><span class="p">),</span> |
| <span class="n">windowedCounts</span><span class="p">.</span><span class="n">word</span> |
| <span class="p">).</span><span class="n">count</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">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="nv">words</span> <span class="k">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">windowedCounts</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="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span> |
| <span class="o">).</span><span class="py">count</span><span class="o">()</span> |
| |
| <span class="c1">// Group the windowed data by another window and word and compute the count of each group</span> |
| <span class="k">val</span> <span class="nv">anotherWindowedCounts</span> <span class="k">=</span> <span class="nv">windowedCounts</span><span class="o">.</span><span class="py">groupBy</span><span class="o">(</span> |
| <span class="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"window"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">),</span> |
| <span class="n">$</span><span class="s">"word"</span> |
| <span class="o">).</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="o">...</span> <span class="c1">// streaming DataFrame of schema { timestamp: Timestamp, word: String }</span> |
| |
| <span class="c1">// Group the data by window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">windowedCounts</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="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">),</span> <span class="s">"10 minutes"</span><span class="o">,</span> <span class="s">"5 minutes"</span><span class="o">),</span> |
| <span class="n">words</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">).</span><span class="na">count</span><span class="o">();</span> |
| |
| <span class="c1">// Group the windowed data by another window and word and compute the count of each group</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">anotherWindowedCounts</span> <span class="o">=</span> <span class="n">windowedCounts</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
| <span class="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="s">"window"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">),</span> |
| <span class="n">windowedCounts</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"word"</span><span class="o">)</span> |
| <span class="o">).</span><span class="na">count</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h5 class="no_toc" id="conditions-for-watermarking-to-clean-aggregation-state">Conditions for watermarking to clean aggregation state</h5> |
| |
| <p>It is important to note that the following conditions must be satisfied for the watermarking to |
| clean the state in aggregation queries <em>(as of Spark 2.1.1, subject to change in the future)</em>.</p> |
| |
| <ul> |
| <li> |
| <p><strong>Output mode must be Append or Update.</strong> Complete mode requires all aggregate data to be preserved, |
| and hence cannot use watermarking to drop intermediate state. See the <a href="#output-modes">Output Modes</a> |
| section for detailed explanation of the semantics of each output mode.</p> |
| </li> |
| <li> |
| <p>The aggregation must have either the event-time column, or a <code class="language-plaintext highlighter-rouge">window</code> on the event-time column.</p> |
| </li> |
| <li> |
| <p><code class="language-plaintext highlighter-rouge">withWatermark</code> must be called on the |
| same column as the timestamp column used in the aggregate. For example, |
| <code class="language-plaintext highlighter-rouge">df.withWatermark("time", "1 min").groupBy("time2").count()</code> is invalid |
| in Append output mode, as watermark is defined on a different column |
| from the aggregation column.</p> |
| </li> |
| <li> |
| <p><code class="language-plaintext highlighter-rouge">withWatermark</code> must be called before the aggregation for the watermark details to be used. |
| For example, <code class="language-plaintext highlighter-rouge">df.groupBy("time").count().withWatermark("time", "1 min")</code> is invalid in Append |
| output mode.</p> |
| </li> |
| </ul> |
| |
| <h5 class="no_toc" id="semantic-guarantees-of-aggregation-with-watermarking">Semantic Guarantees of Aggregation with Watermarking</h5> |
| |
| <ul> |
| <li> |
| <p>A watermark delay (set with <code class="language-plaintext highlighter-rouge">withWatermark</code>) of “2 hours” guarantees that the engine will never |
| drop any data that is less than 2 hours delayed. In other words, any data less than 2 hours behind |
| (in terms of event-time) the latest data processed till then is guaranteed to be aggregated.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| </ul> |
| |
| <h3 id="join-operations">Join Operations</h3> |
| <p>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.</p> |
| |
| <h4 id="stream-static-joins">Stream-static Joins</h4> |
| |
| <p>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.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">staticDf</span> <span class="o">=</span> <span class="n">spark</span><span class="p">.</span><span class="n">read</span><span class="p">.</span> <span class="p">...</span> |
| <span class="n">streamingDf</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="p">...</span> |
| <span class="n">streamingDf</span><span class="p">.</span><span class="n">join</span><span class="p">(</span><span class="n">staticDf</span><span class="p">,</span> <span class="s">"type"</span><span class="p">)</span> <span class="c1"># inner equi-join with a static DF |
| </span><span class="n">streamingDf</span><span class="p">.</span><span class="n">join</span><span class="p">(</span><span class="n">staticDf</span><span class="p">,</span> <span class="s">"type"</span><span class="p">,</span> <span class="s">"left_outer"</span><span class="p">)</span> <span class="c1"># left outer join with a static DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">staticDf</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">read</span><span class="o">.</span> <span class="o">...</span> |
| <span class="k">val</span> <span class="nv">streamingDf</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="o">...</span> |
| |
| <span class="nv">streamingDf</span><span class="o">.</span><span class="py">join</span><span class="o">(</span><span class="n">staticDf</span><span class="o">,</span> <span class="s">"type"</span><span class="o">)</span> <span class="c1">// inner equi-join with a static DF</span> |
| <span class="nv">streamingDf</span><span class="o">.</span><span class="py">join</span><span class="o">(</span><span class="n">staticDf</span><span class="o">,</span> <span class="s">"type"</span><span class="o">,</span> <span class="s">"left_outer"</span><span class="o">)</span> <span class="c1">// left outer join with a static DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">staticDf</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span> <span class="o">...;</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">streamingDf</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="n">streamingDf</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">staticDf</span><span class="o">,</span> <span class="s">"type"</span><span class="o">);</span> <span class="c1">// inner equi-join with a static DF</span> |
| <span class="n">streamingDf</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">staticDf</span><span class="o">,</span> <span class="s">"type"</span><span class="o">,</span> <span class="s">"left_outer"</span><span class="o">);</span> <span class="c1">// left outer join with a static DF</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">staticDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.df</span><span class="p">(</span><span class="n">...</span><span class="p">)</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="n">...</span><span class="p">)</span><span class="w"> |
| </span><span class="n">joined</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">merge</span><span class="p">(</span><span class="n">streamingDf</span><span class="p">,</span><span class="w"> </span><span class="n">staticDf</span><span class="p">,</span><span class="w"> </span><span class="n">sort</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="kc">FALSE</span><span class="p">)</span><span class="w"> </span><span class="c1"># inner equi-join with a static DF</span><span class="w"> |
| </span><span class="n">joined</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">join</span><span class="p">(</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="p">,</span><span class="w"> |
| </span><span class="n">staticDf</span><span class="p">,</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="o">$</span><span class="n">value</span><span class="w"> </span><span class="o">==</span><span class="w"> </span><span class="n">staticDf</span><span class="o">$</span><span class="n">value</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"left_outer"</span><span class="p">)</span><span class="w"> </span><span class="c1"># left outer join with a static DF</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>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 <a href="#support-matrix-for-joins-in-streaming-queries">end of this Join section</a>.</p> |
| |
| <h4 id="stream-stream-joins">Stream-stream Joins</h4> |
| <p>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.</p> |
| |
| <h5 id="inner-joins-with-optional-watermarking">Inner Joins with optional Watermarking</h5> |
| <p>Inner joins on any kind of columns along with any kind of join conditions are supported. |
| However, as the stream runs, the size of streaming state will keep growing indefinitely as |
| <em>all</em> past input must be saved as any new input can match with any input from the past. |
| To avoid unbounded state, you have to define additional join conditions such that indefinitely |
| old inputs cannot match with future inputs and therefore can be cleared from the state. |
| In other words, you will have to do the following additional steps in the join.</p> |
| |
| <ol> |
| <li> |
| <p>Define watermark delays on both inputs such that the engine knows how delayed the input can be |
| (similar to streaming aggregations)</p> |
| </li> |
| <li> |
| <p>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.</p> |
| |
| <ol> |
| <li> |
| <p>Time range join conditions (e.g. <code class="language-plaintext highlighter-rouge">...JOIN ON leftTime BETWEEN rightTime AND rightTime + INTERVAL 1 HOUR</code>),</p> |
| </li> |
| <li> |
| <p>Join on event-time windows (e.g. <code class="language-plaintext highlighter-rouge">...JOIN ON leftTimeWindow = rightTimeWindow</code>).</p> |
| </li> |
| </ol> |
| </li> |
| </ol> |
| |
| <p>Let’s understand this with an example.</p> |
| |
| <p>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.</p> |
| |
| <ol> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>Event-time range condition: Say, a click can occur within a time range of 0 seconds to 1 hour |
| after the corresponding impression.</p> |
| </li> |
| </ol> |
| |
| <p>The code would look like this.</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="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">expr</span> |
| |
| <span class="n">impressions</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="p">...</span> |
| <span class="n">clicks</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="p">...</span> |
| |
| <span class="c1"># Apply watermarks on event-time columns |
| </span><span class="n">impressionsWithWatermark</span> <span class="o">=</span> <span class="n">impressions</span><span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"impressionTime"</span><span class="p">,</span> <span class="s">"2 hours"</span><span class="p">)</span> |
| <span class="n">clicksWithWatermark</span> <span class="o">=</span> <span class="n">clicks</span><span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"clickTime"</span><span class="p">,</span> <span class="s">"3 hours"</span><span class="p">)</span> |
| |
| <span class="c1"># Join with event-time constraints |
| </span><span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">join</span><span class="p">(</span> |
| <span class="n">clicksWithWatermark</span><span class="p">,</span> |
| <span class="n">expr</span><span class="p">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="p">)</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.expr</span> |
| |
| <span class="k">val</span> <span class="nv">impressions</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="o">...</span> |
| <span class="k">val</span> <span class="nv">clicks</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="o">...</span> |
| |
| <span class="c1">// Apply watermarks on event-time columns</span> |
| <span class="k">val</span> <span class="nv">impressionsWithWatermark</span> <span class="k">=</span> <span class="nv">impressions</span><span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"impressionTime"</span><span class="o">,</span> <span class="s">"2 hours"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="nv">clicksWithWatermark</span> <span class="k">=</span> <span class="nv">clicks</span><span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"clickTime"</span><span class="o">,</span> <span class="s">"3 hours"</span><span class="o">)</span> |
| |
| <span class="c1">// Join with event-time constraints</span> |
| <span class="nv">impressionsWithWatermark</span><span class="o">.</span><span class="py">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="nf">expr</span><span class="o">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="o">)</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="kn">import</span> <span class="nn">static</span> <span class="n">org</span><span class="o">.</span><span class="na">apache</span><span class="o">.</span><span class="na">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">.</span><span class="na">functions</span><span class="o">.</span><span class="na">expr</span> |
| |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">impressions</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">clicks</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="c1">// Apply watermarks on event-time columns</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">impressionsWithWatermark</span> <span class="o">=</span> <span class="n">impressions</span><span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"impressionTime"</span><span class="o">,</span> <span class="s">"2 hours"</span><span class="o">);</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">clicksWithWatermark</span> <span class="o">=</span> <span class="n">clicks</span><span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"clickTime"</span><span class="o">,</span> <span class="s">"3 hours"</span><span class="o">);</span> |
| |
| <span class="c1">// Join with event-time constraints</span> |
| <span class="n">impressionsWithWatermark</span><span class="o">.</span><span class="na">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="n">expr</span><span class="o">(</span> |
| <span class="s">"clickAdId = impressionAdId AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime >= impressionTime AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime <= impressionTime + interval 1 hour "</span><span class="o">)</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">impressions</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="n">...</span><span class="p">)</span><span class="w"> |
| </span><span class="n">clicks</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="n">...</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Apply watermarks on event-time columns</span><span class="w"> |
| </span><span class="n">impressionsWithWatermark</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">withWatermark</span><span class="p">(</span><span class="n">impressions</span><span class="p">,</span><span class="w"> </span><span class="s2">"impressionTime"</span><span class="p">,</span><span class="w"> </span><span class="s2">"2 hours"</span><span class="p">)</span><span class="w"> |
| </span><span class="n">clicksWithWatermark</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">withWatermark</span><span class="p">(</span><span class="n">clicks</span><span class="p">,</span><span class="w"> </span><span class="s2">"clickTime"</span><span class="p">,</span><span class="w"> </span><span class="s2">"3 hours"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Join with event-time constraints</span><span class="w"> |
| </span><span class="n">joined</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">join</span><span class="p">(</span><span class="w"> |
| </span><span class="n">impressionsWithWatermark</span><span class="p">,</span><span class="w"> |
| </span><span class="n">clicksWithWatermark</span><span class="p">,</span><span class="w"> |
| </span><span class="n">expr</span><span class="p">(</span><span class="w"> |
| </span><span class="n">paste</span><span class="p">(</span><span class="w"> |
| </span><span class="s2">"clickAdId = impressionAdId AND"</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"clickTime >= impressionTime AND"</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"clickTime <= impressionTime + interval 1 hour"</span><span class="w"> |
| </span><span class="p">)))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h6 class="no_toc" id="semantic-guarantees-of-stream-stream-inner-joins-with-watermarking">Semantic Guarantees of Stream-stream Inner Joins with Watermarking</h6> |
| <p>This is similar to the <a href="#semantic-guarantees-of-aggregation-with-watermarking">guarantees provided by watermarking on aggregations</a>. |
| A watermark delay of “2 hours” guarantees that the engine will never drop any data that is less than |
| 2 hours delayed. But data delayed by more than 2 hours may or may not get processed.</p> |
| |
| <h5 id="outer-joins-with-watermarking">Outer Joins with Watermarking</h5> |
| <p>While the watermark + event-time constraints is optional for inner joins, for outer joins |
| they must be specified. This is because for generating the NULL results in outer join, the |
| engine must know when an input row is not going to match with anything in future. Hence, the |
| watermark + event-time constraints must be specified for generating correct results. Therefore, |
| a query with outer-join will look quite like the ad-monetization example earlier, except that |
| there will be an additional parameter specifying it to be an outer-join.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">join</span><span class="p">(</span> |
| <span class="n">clicksWithWatermark</span><span class="p">,</span> |
| <span class="n">expr</span><span class="p">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="p">),</span> |
| <span class="s">"leftOuter"</span> <span class="c1"># can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi" |
| </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="nv">impressionsWithWatermark</span><span class="o">.</span><span class="py">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="nf">expr</span><span class="o">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="o">),</span> |
| <span class="n">joinType</span> <span class="k">=</span> <span class="s">"leftOuter"</span> <span class="c1">// can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"</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="n">impressionsWithWatermark</span><span class="o">.</span><span class="na">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="n">expr</span><span class="o">(</span> |
| <span class="s">"clickAdId = impressionAdId AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime >= impressionTime AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime <= impressionTime + interval 1 hour "</span><span class="o">),</span> |
| <span class="s">"leftOuter"</span> <span class="c1">// can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"</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">joined</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">join</span><span class="p">(</span><span class="w"> |
| </span><span class="n">impressionsWithWatermark</span><span class="p">,</span><span class="w"> |
| </span><span class="n">clicksWithWatermark</span><span class="p">,</span><span class="w"> |
| </span><span class="n">expr</span><span class="p">(</span><span class="w"> |
| </span><span class="n">paste</span><span class="p">(</span><span class="w"> |
| </span><span class="s2">"clickAdId = impressionAdId AND"</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"clickTime >= impressionTime AND"</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"clickTime <= impressionTime + interval 1 hour"</span><span class="p">),</span><span class="w"> |
| </span><span class="s2">"left_outer"</span><span class="w"> </span><span class="c1"># can be "inner", "left_outer", "right_outer", "full_outer", "left_semi"</span><span class="w"> |
| </span><span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h6 class="no_toc" id="semantic-guarantees-of-stream-stream-outer-joins-with-watermarking">Semantic Guarantees of Stream-stream Outer Joins with Watermarking</h6> |
| <p>Outer joins have the same guarantees as <a href="#semantic-guarantees-of-stream-stream-inner-joins-with-watermarking">inner joins</a> |
| regarding watermark delays and whether data will be dropped or not.</p> |
| |
| <h6 class="no_toc" id="caveats">Caveats</h6> |
| <p>There are a few important characteristics to note regarding how the outer results are generated.</p> |
| |
| <ul> |
| <li> |
| <p><em>The outer NULL results will be generated with a delay that depends on the specified watermark |
| delay and the time range condition.</em> 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.</p> |
| </li> |
| <li> |
| <p>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. |
| <em>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.</em></p> |
| </li> |
| </ul> |
| |
| <h5 id="semi-joins-with-watermarking">Semi Joins with Watermarking</h5> |
| <p>A semi join returns values from the left side of the relation that has a match with the right. |
| It is also referred to as a left semi join. Similar to outer joins, watermark + event-time |
| constraints must be specified for semi join. This is to evict unmatched input rows on left side, |
| the engine must know when an input row on left side is not going to match with anything on right |
| side in future.</p> |
| |
| <h6 class="no_toc" id="semantic-guarantees-of-stream-stream-semi-joins-with-watermarking">Semantic Guarantees of Stream-stream Semi Joins with Watermarking</h6> |
| <p>Semi joins have the same guarantees as <a href="#semantic-guarantees-of-stream-stream-inner-joins-with-watermarking">inner joins</a> |
| regarding watermark delays and whether data will be dropped or not.</p> |
| |
| <h5 id="support-matrix-for-joins-in-streaming-queries">Support matrix for joins in streaming queries</h5> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Left Input</th> |
| <th>Right Input</th> |
| <th>Join Type</th> |
| <th></th> |
| </tr> |
| </thead> |
| <tr> |
| <td style="vertical-align: middle;">Static</td> |
| <td style="vertical-align: middle;">Static</td> |
| <td style="vertical-align: middle;">All types</td> |
| <td style="vertical-align: middle;"> |
| Supported, since its not on streaming data even though it |
| can be present in a streaming query |
| </td> |
| </tr> |
| <tr> |
| <td rowspan="5" style="vertical-align: middle;">Stream</td> |
| <td rowspan="5" style="vertical-align: middle;">Static</td> |
| <td style="vertical-align: middle;">Inner</td> |
| <td style="vertical-align: middle;">Supported, not stateful</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Outer</td> |
| <td style="vertical-align: middle;">Supported, not stateful</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Right Outer</td> |
| <td style="vertical-align: middle;">Not supported</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Full Outer</td> |
| <td style="vertical-align: middle;">Not supported</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Semi</td> |
| <td style="vertical-align: middle;">Supported, not stateful</td> |
| </tr> |
| <tr> |
| <td rowspan="5" style="vertical-align: middle;">Static</td> |
| <td rowspan="5" style="vertical-align: middle;">Stream</td> |
| <td style="vertical-align: middle;">Inner</td> |
| <td style="vertical-align: middle;">Supported, not stateful</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Outer</td> |
| <td style="vertical-align: middle;">Not supported</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Right Outer</td> |
| <td style="vertical-align: middle;">Supported, not stateful</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Full Outer</td> |
| <td style="vertical-align: middle;">Not supported</td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Semi</td> |
| <td style="vertical-align: middle;">Not supported</td> |
| </tr> |
| <tr> |
| <td rowspan="5" style="vertical-align: middle;">Stream</td> |
| <td rowspan="5" style="vertical-align: middle;">Stream</td> |
| <td style="vertical-align: middle;">Inner</td> |
| <td style="vertical-align: middle;"> |
| Supported, optionally specify watermark on both sides + |
| time constraints for state cleanup |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Outer</td> |
| <td style="vertical-align: middle;"> |
| Conditionally supported, must specify watermark on right + time constraints for correct |
| results, optionally specify watermark on left for all state cleanup |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Right Outer</td> |
| <td style="vertical-align: middle;"> |
| Conditionally supported, must specify watermark on left + time constraints for correct |
| results, optionally specify watermark on right for all state cleanup |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Full Outer</td> |
| <td style="vertical-align: middle;"> |
| Conditionally supported, must specify watermark on one side + time constraints for correct |
| results, optionally specify watermark on the other side for all state cleanup |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Left Semi</td> |
| <td style="vertical-align: middle;"> |
| Conditionally supported, must specify watermark on right + time constraints for correct |
| results, optionally specify watermark on left for all state cleanup |
| </td> |
| </tr> |
| <tr> |
| <td></td> |
| <td></td> |
| <td></td> |
| <td></td> |
| </tr> |
| </table> |
| |
| <p>Additional details on supported joins:</p> |
| |
| <ul> |
| <li> |
| <p>Joins can be cascaded, that is, you can do <code class="language-plaintext highlighter-rouge">df1.join(df2, ...).join(df3, ...).join(df4, ....)</code>.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>You cannot use mapGroupsWithState and flatMapGroupsWithState before and after joins.</p> |
| </li> |
| </ul> |
| |
| <p>In append output mode, you can construct a query having non-map-like operations e.g. aggregation, deduplication, stream-stream join before/after join.</p> |
| |
| <p>For example, here’s an example of time window aggregation in both streams followed by stream-stream join with event time window:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">clicksWindow</span> <span class="k">=</span> <span class="n">clicksWithWatermark</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="nf">window</span><span class="o">(</span><span class="s">"clickTime"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="py">count</span><span class="o">()</span> |
| |
| <span class="k">val</span> <span class="nv">impressionsWindow</span> <span class="k">=</span> <span class="n">impressionsWithWatermark</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="nf">window</span><span class="o">(</span><span class="s">"impressionTime"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="py">count</span><span class="o">()</span> |
| |
| <span class="nv">clicksWindow</span><span class="o">.</span><span class="py">join</span><span class="o">(</span><span class="n">impressionsWindow</span><span class="o">,</span> <span class="s">"window"</span><span class="o">,</span> <span class="s">"inner"</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">clicksWindow</span> <span class="o">=</span> <span class="n">clicksWithWatermark</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">clicksWithWatermark</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"clickTime"</span><span class="o">),</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</span><span class="o">();</span> |
| |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">impressionsWindow</span> <span class="o">=</span> <span class="n">impressionsWithWatermark</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">impressionsWithWatermark</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"impressionTime"</span><span class="o">),</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</span><span class="o">();</span> |
| |
| <span class="n">clicksWindow</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">impressionsWindow</span><span class="o">,</span> <span class="s">"window"</span><span class="o">,</span> <span class="s">"inner"</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">clicksWindow</span> <span class="o">=</span> <span class="n">clicksWithWatermark</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">clicksWithWatermark</span><span class="p">.</span><span class="n">clickAdId</span><span class="p">,</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">clicksWithWatermark</span><span class="p">.</span><span class="n">clickTime</span><span class="p">,</span> <span class="s">"1 hour"</span><span class="p">)</span> |
| <span class="p">).</span><span class="n">count</span><span class="p">()</span> |
| |
| <span class="n">impressionsWindow</span> <span class="o">=</span> <span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">impressionAdId</span><span class="p">,</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">impressionTime</span><span class="p">,</span> <span class="s">"1 hour"</span><span class="p">)</span> |
| <span class="p">).</span><span class="n">count</span><span class="p">()</span> |
| |
| <span class="n">clicksWindow</span><span class="p">.</span><span class="n">join</span><span class="p">(</span><span class="n">impressionsWindow</span><span class="p">,</span> <span class="s">"window"</span><span class="p">,</span> <span class="s">"inner"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <p>Here’s another example of stream-stream join with time range join condition followed by time window aggregation:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">joined</span> <span class="k">=</span> <span class="nv">impressionsWithWatermark</span><span class="o">.</span><span class="py">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="nf">expr</span><span class="o">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="o">),</span> |
| <span class="n">joinType</span> <span class="k">=</span> <span class="s">"leftOuter"</span> <span class="c1">// can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"</span> |
| <span class="o">)</span> |
| |
| <span class="n">joined</span> |
| <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="n">$</span><span class="s">"clickAdId"</span><span class="o">,</span> <span class="nf">window</span><span class="o">(</span><span class="n">$</span><span class="s">"clickTime"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="py">count</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">impressionsWithWatermark</span><span class="o">.</span><span class="na">join</span><span class="o">(</span> |
| <span class="n">clicksWithWatermark</span><span class="o">,</span> |
| <span class="n">expr</span><span class="o">(</span> |
| <span class="s">"clickAdId = impressionAdId AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime >= impressionTime AND "</span> <span class="o">+</span> |
| <span class="s">"clickTime <= impressionTime + interval 1 hour "</span><span class="o">),</span> |
| <span class="s">"leftOuter"</span> <span class="c1">// can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"</span> |
| <span class="o">);</span> |
| |
| <span class="n">joined</span> |
| <span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="n">joined</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"clickAdId"</span><span class="o">),</span> <span class="n">functions</span><span class="o">.</span><span class="na">window</span><span class="o">(</span><span class="n">joined</span><span class="o">.</span><span class="na">col</span><span class="o">(</span><span class="s">"clickTime"</span><span class="o">),</span> <span class="s">"1 hour"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">count</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">joined</span> <span class="o">=</span> <span class="n">impressionsWithWatermark</span><span class="p">.</span><span class="n">join</span><span class="p">(</span> |
| <span class="n">clicksWithWatermark</span><span class="p">,</span> |
| <span class="n">expr</span><span class="p">(</span><span class="s">""" |
| clickAdId = impressionAdId AND |
| clickTime >= impressionTime AND |
| clickTime <= impressionTime + interval 1 hour |
| """</span><span class="p">),</span> |
| <span class="s">"leftOuter"</span> <span class="c1"># can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi" |
| </span><span class="p">)</span> |
| |
| <span class="n">joined</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span> |
| <span class="n">joined</span><span class="p">.</span><span class="n">clickAdId</span><span class="p">,</span> |
| <span class="n">window</span><span class="p">(</span><span class="n">joined</span><span class="p">.</span><span class="n">clickTime</span><span class="p">,</span> <span class="s">"1 hour"</span><span class="p">)</span> |
| <span class="p">).</span><span class="n">count</span><span class="p">()</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <h3 id="streaming-deduplication">Streaming Deduplication</h3> |
| <p>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.</p> |
| |
| <ul> |
| <li> |
| <p><em>With watermark</em> - 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.</p> |
| </li> |
| <li> |
| <p><em>Without watermark</em> - Since there are no bounds on when a duplicate record may arrive, the query stores the data from all the past records as state.</p> |
| </li> |
| </ul> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">streamingDf</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="p">...</span> |
| |
| <span class="c1"># Without watermark using guid column |
| </span><span class="n">streamingDf</span><span class="p">.</span><span class="n">dropDuplicates</span><span class="p">(</span><span class="s">"guid"</span><span class="p">)</span> |
| |
| <span class="c1"># With watermark using guid and eventTime columns |
| </span><span class="n">streamingDf</span> \ |
| <span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"eventTime"</span><span class="p">,</span> <span class="s">"10 seconds"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">dropDuplicates</span><span class="p">(</span><span class="s">"guid"</span><span class="p">,</span> <span class="s">"eventTime"</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">val</span> <span class="nv">streamingDf</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="o">...</span> <span class="c1">// columns: guid, eventTime, ...</span> |
| |
| <span class="c1">// Without watermark using guid column</span> |
| <span class="nv">streamingDf</span><span class="o">.</span><span class="py">dropDuplicates</span><span class="o">(</span><span class="s">"guid"</span><span class="o">)</span> |
| |
| <span class="c1">// With watermark using guid and eventTime columns</span> |
| <span class="n">streamingDf</span> |
| <span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"eventTime"</span><span class="o">,</span> <span class="s">"10 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">dropDuplicates</span><span class="o">(</span><span class="s">"guid"</span><span class="o">,</span> <span class="s">"eventTime"</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">streamingDf</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="c1">// columns: guid, eventTime, ...</span> |
| |
| <span class="c1">// Without watermark using guid column</span> |
| <span class="n">streamingDf</span><span class="o">.</span><span class="na">dropDuplicates</span><span class="o">(</span><span class="s">"guid"</span><span class="o">);</span> |
| |
| <span class="c1">// With watermark using guid and eventTime columns</span> |
| <span class="n">streamingDf</span> |
| <span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"eventTime"</span><span class="o">,</span> <span class="s">"10 seconds"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">dropDuplicates</span><span class="o">(</span><span class="s">"guid"</span><span class="o">,</span> <span class="s">"eventTime"</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">streamingDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.stream</span><span class="p">(</span><span class="n">...</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Without watermark using guid column</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">dropDuplicates</span><span class="p">(</span><span class="n">streamingDf</span><span class="p">,</span><span class="w"> </span><span class="s2">"guid"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># With watermark using guid and eventTime columns</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">withWatermark</span><span class="p">(</span><span class="n">streamingDf</span><span class="p">,</span><span class="w"> </span><span class="s2">"eventTime"</span><span class="p">,</span><span class="w"> </span><span class="s2">"10 seconds"</span><span class="p">)</span><span class="w"> |
| </span><span class="n">streamingDf</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">dropDuplicates</span><span class="p">(</span><span class="n">streamingDf</span><span class="p">,</span><span class="w"> </span><span class="s2">"guid"</span><span class="p">,</span><span class="w"> </span><span class="s2">"eventTime"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>Specifically for streaming, you can deduplicate records in data streams using a unique identifier in the events, within the time range of watermark. |
| For example, if you set the delay threshold of watermark as “1 hour”, duplicated events which occurred within 1 hour can be correctly deduplicated. |
| (For more details, please refer to the API doc of <a href="/api/scala/org/apache/spark/sql/Dataset.html#dropDuplicatesWithinWatermark():org.apache.spark.sql.Dataset[T]">dropDuplicatesWithinWatermark</a>.)</p> |
| |
| <p>This can be used to deal with use case where event time column cannot be a part of unique identifier, mostly due to the case |
| where event times are somehow different for the same records. (E.g. non-idempotent writer where issuing event time happens at write)</p> |
| |
| <p>Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.</p> |
| |
| <p>This feature requires watermark with delay threshold to be set in streaming DataFrame/Dataset.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">streamingDf</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="p">...</span> |
| |
| <span class="c1"># deduplicate using guid column with watermark based on eventTime column |
| </span><span class="n">streamingDf</span> \ |
| <span class="p">.</span><span class="n">withWatermark</span><span class="p">(</span><span class="s">"eventTime"</span><span class="p">,</span> <span class="s">"10 hours"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">dropDuplicatesWithinWatermark</span><span class="p">(</span><span class="s">"guid"</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">val</span> <span class="nv">streamingDf</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="o">...</span> <span class="c1">// columns: guid, eventTime, ...</span> |
| |
| <span class="c1">// deduplicate using guid column with watermark based on eventTime column</span> |
| <span class="n">streamingDf</span> |
| <span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"eventTime"</span><span class="o">,</span> <span class="s">"10 hours"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">dropDuplicatesWithinWatermark</span><span class="o">(</span><span class="s">"guid"</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="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">streamingDf</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="c1">// columns: guid, eventTime, ...</span> |
| |
| <span class="c1">// deduplicate using guid column with watermark based on eventTime column</span> |
| <span class="n">streamingDf</span> |
| <span class="o">.</span><span class="na">withWatermark</span><span class="o">(</span><span class="s">"eventTime"</span><span class="o">,</span> <span class="s">"10 hours"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">dropDuplicatesWithinWatermark</span><span class="o">(</span><span class="s">"guid"</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="policy-for-handling-multiple-watermarks">Policy for handling multiple watermarks</h3> |
| <p>A streaming query can have multiple input streams that are unioned or joined together. |
| Each of the input streams can have a different threshold of late data that needs to |
| be tolerated for stateful operations. You specify these thresholds using |
| <code class="language-plaintext highlighter-rouge">withWatermarks("eventTime", delay)</code> on each of the input streams. For example, consider |
| a query with stream-stream joins between <code class="language-plaintext highlighter-rouge">inputStream1</code> and <code class="language-plaintext highlighter-rouge">inputStream2</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="nv">inputStream1</span><span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"eventTime1"</span><span class="o">,</span> <span class="s">"1 hour"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">join</span><span class="o">(</span> |
| <span class="nv">inputStream2</span><span class="o">.</span><span class="py">withWatermark</span><span class="o">(</span><span class="s">"eventTime2"</span><span class="o">,</span> <span class="s">"2 hours"</span><span class="o">),</span> |
| <span class="n">joinCondition</span><span class="o">)</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <p>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.</p> |
| |
| <p>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 |
| <code class="language-plaintext highlighter-rouge">spark.sql.streaming.multipleWatermarkPolicy</code> to <code class="language-plaintext highlighter-rouge">max</code> (default is <code class="language-plaintext highlighter-rouge">min</code>). |
| 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.</p> |
| |
| <h3 id="arbitrary-stateful-operations">Arbitrary Stateful Operations</h3> |
| <p>Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation <code class="language-plaintext highlighter-rouge">mapGroupsWithState</code> and the more powerful operation <code class="language-plaintext highlighter-rouge">flatMapGroupsWithState</code>. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation (<a href="api/scala/org/apache/spark/sql/streaming/GroupState.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/GroupState.html">Java</a>) and the examples (<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredComplexSessionization.scala">Scala</a>/<a href="https://github.com/apache/spark/blob/v3.5.0/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredComplexSessionization.java">Java</a>).</p> |
| |
| <p>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.</p> |
| |
| <h3 id="unsupported-operations">Unsupported Operations</h3> |
| <p>There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets. |
| Some of them are as follows.</p> |
| |
| <ul> |
| <li> |
| <p>Limit and take the first N rows are not supported on streaming Datasets.</p> |
| </li> |
| <li> |
| <p>Distinct operations on streaming Datasets are not supported.</p> |
| </li> |
| <li> |
| <p>Sorting operations are supported on streaming Datasets only after an aggregation and in Complete Output Mode.</p> |
| </li> |
| <li> |
| <p>Few types of outer joins on streaming Datasets are not supported. See the |
| <a href="#support-matrix-for-joins-in-streaming-queries">support matrix in the Join Operations section</a> |
| for more details.</p> |
| </li> |
| <li> |
| <p>Chaining multiple stateful operations on streaming Datasets is not supported with Update and Complete mode.</p> |
| <ul> |
| <li>In addition, mapGroupsWithState/flatMapGroupsWithState operation followed by other stateful operation is not supported in Append mode.</li> |
| <li>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.</li> |
| </ul> |
| </li> |
| </ul> |
| |
| <p>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).</p> |
| |
| <ul> |
| <li> |
| <p><code class="language-plaintext highlighter-rouge">count()</code> - Cannot return a single count from a streaming Dataset. Instead, use <code class="language-plaintext highlighter-rouge">ds.groupBy().count()</code> which returns a streaming Dataset containing a running count.</p> |
| </li> |
| <li> |
| <p><code class="language-plaintext highlighter-rouge">foreach()</code> - Instead use <code class="language-plaintext highlighter-rouge">ds.writeStream.foreach(...)</code> (see next section).</p> |
| </li> |
| <li> |
| <p><code class="language-plaintext highlighter-rouge">show()</code> - Instead use the console sink (see next section).</p> |
| </li> |
| </ul> |
| |
| <p>If you try any of these operations, you will see an <code class="language-plaintext highlighter-rouge">AnalysisException</code> 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.</p> |
| |
| <h3 id="state-store">State Store</h3> |
| |
| <p>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.</p> |
| |
| <h4 id="hdfs-state-store-provider">HDFS state store provider</h4> |
| |
| <p>The HDFS backend state store provider is the default implementation of [[StateStoreProvider]] and |
| [[StateStore]] in which all the data is stored in memory map in the first stage, and then backed |
| by files in an HDFS-compatible file system. All updates to the store have to be done in sets |
| transactionally, and each set of updates increments the store’s version. These versions can be |
| used to re-execute the updates (by retries in RDD operations) on the correct version of the store, |
| and regenerate the store version.</p> |
| |
| <h4 id="rocksdb-state-store-implementation">RocksDB state store implementation</h4> |
| |
| <p>As of Spark 3.2, we add a new built-in state store implementation, RocksDB state store provider.</p> |
| |
| <p>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.</p> |
| |
| <p>In such cases, you can choose to use a more optimized state management solution based on |
| <a href="https://rocksdb.org/">RocksDB</a>. 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).</p> |
| |
| <p>To enable the new build-in state store implementation, set <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.providerClass</code> |
| to <code class="language-plaintext highlighter-rouge">org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider</code>.</p> |
| |
| <p>Here are the configs regarding to RocksDB instance of the state store provider:</p> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Config Name</th> |
| <th>Description</th> |
| <th>Default Value</th> |
| </tr> |
| </thead> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.compactOnCommit</td> |
| <td>Whether we perform a range compaction of RocksDB instance for commit operation</td> |
| <td>False</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled</td> |
| <td>Whether to upload changelog instead of snapshot during RocksDB StateStore commit</td> |
| <td>False</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.blockSizeKB</td> |
| <td>Approximate size in KB of user data packed per block for a RocksDB BlockBasedTable, which is a RocksDB's default SST file format.</td> |
| <td>4</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.blockCacheSizeMB</td> |
| <td>The size capacity in MB for a cache of blocks.</td> |
| <td>8</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.lockAcquireTimeoutMs</td> |
| <td>The waiting time in millisecond for acquiring lock in the load operation for RocksDB instance.</td> |
| <td>60000</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.maxOpenFiles</td> |
| <td>The number of open files that can be used by the RocksDB 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.</td> |
| <td>-1</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.resetStatsOnLoad</td> |
| <td>Whether we resets all ticker and histogram stats for RocksDB on load.</td> |
| <td>True</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.trackTotalNumberOfRows</td> |
| <td>Whether we track the total number of rows in state store. Please refer the details in <a href="#performance-aspect-considerations">Performance-aspect considerations</a>.</td> |
| <td>True</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.writeBufferSizeMB</td> |
| <td>The maximum size of MemTable in RocksDB. Value of -1 means that RocksDB internal default values will be used</td> |
| <td>-1</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber</td> |
| <td>The maximum number of MemTables in RocksDB, both active and immutable. Value of -1 means that RocksDB internal default values will be used</td> |
| <td>-1</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.boundedMemoryUsage</td> |
| <td>Whether total memory usage for RocksDB state store instances on a single node is bounded.</td> |
| <td>false</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.maxMemoryUsageMB</td> |
| <td>Total memory limit in MB for RocksDB state store instances on a single node.</td> |
| <td>500</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.writeBufferCacheRatio</td> |
| <td>Total memory to be occupied by write buffers as a fraction of memory allocated across all RocksDB instances on a single node using maxMemoryUsageMB.</td> |
| <td>0.5</td> |
| </tr> |
| <tr> |
| <td>spark.sql.streaming.stateStore.rocksdb.highPriorityPoolRatio</td> |
| <td>Total memory to be occupied by blocks in high priority pool as a fraction of memory allocated across all RocksDB instances on a single node using maxMemoryUsageMB.</td> |
| <td>0.1</td> |
| </tr> |
| </table> |
| |
| <h5 id="rocksdb-state-store-memory-management">RocksDB State Store Memory Management</h5> |
| <p>RocksDB allocates memory for different objects such as memtables, block cache and filter/index blocks. If left unbounded, RocksDB memory usage across multiple instances could grow indefinitely and potentially cause OOM (out-of-memory) issues. |
| RocksDB provides a way to limit the memory usage for all DB instances running on a single node by using the write buffer manager functionality. |
| If you want to cap RocksDB memory usage in your Spark Structured Streaming deployment, this feature can be enabled by setting the <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.rocksdb.boundedMemoryUsage</code> config to <code class="language-plaintext highlighter-rouge">true</code>. |
| You can also determine the max allowed memory for RocksDB instances by setting the <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.rocksdb.maxMemoryUsageMB</code> value to a static number or as a fraction of the physical memory available on the node. |
| Limits for individual RocksDB instances can also be configured by setting <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.rocksdb.writeBufferSizeMB</code> and <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber</code> to the required values. By default, RocksDB internal defaults are used for these settings.</p> |
| |
| <h5 id="rocksdb-state-store-changelog-checkpointing">RocksDB State Store Changelog Checkpointing</h5> |
| <p>In newer version of Spark, changelog checkpointing is introduced for RocksDB state store. The traditional checkpointing mechanism for RocksDB State Store is incremental snapshot checkpointing, where the manifest files and newly generated RocksDB SST files of RocksDB instances are uploaded to a durable storage. |
| Instead of uploading data files of RocksDB instances, changelog checkpointing uploads changes made to the state since the last checkpoint for durability. |
| Snapshots are persisted periodically in the background for predictable failure recovery and changelog trimming. |
| Changelog checkpointing avoids cost of capturing and uploading snapshots of RocksDB instances and significantly reduce streaming query latency.</p> |
| |
| <p>Changelog checkpointing is disabled by default. You can enable RocksDB State Store changelog checkpointing by setting <code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled</code> config to <code class="language-plaintext highlighter-rouge">true</code>. |
| Changelog checkpointing is designed to be backward compatible with traditional checkpointing mechanism. |
| RocksDB state store provider offers seamless support for transitioning between two checkpointing mechanisms in both directions. This allows you to leverage the performance benefits of changelog checkpointing without discarding the old state checkpoint. |
| In a version of spark that supports changelog checkpointing, you can migrate streaming queries from older versions of Spark to changelog checkpointing by enabling changelog checkpointing in the spark session. |
| Vice versa, you can disable changelog checkpointing safely in newer version of Spark, then any query that already run with changelog checkpointing will switch back to traditional checkpointing. |
| You would need to restart you streaming queries for change in checkpointing mechanism to be applied, but you won’t observe any performance degrade in the process.</p> |
| |
| <h5 id="performance-aspect-considerations">Performance-aspect considerations</h5> |
| |
| <ol> |
| <li>You may want to disable the track of total number of rows to aim the better performance on RocksDB state store.</li> |
| </ol> |
| |
| <p>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 - <code class="language-plaintext highlighter-rouge">numRowsUpdated</code>, <code class="language-plaintext highlighter-rouge">numRowsRemoved</code>.</p> |
| |
| <p>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 (<code class="language-plaintext highlighter-rouge">numTotalStateRows</code>) will be reported as 0.</p> |
| |
| <h4 id="state-store-and-task-locality">State Store and task locality</h4> |
| |
| <p>The stateful operations store states for events in state stores of executors. State stores occupy resources such as memory and disk space to store the states. |
| So it is more efficient to keep a state store provider running in the same executor across different streaming batches. |
| Changing the location of a state store provider requires the extra overhead of loading checkpointed states. The overhead of loading state from checkpoint depends |
| on the external storage and the size of the state, which tends to hurt the latency of micro-batch run. For some use cases such as processing very large state data, |
| loading new state store providers from checkpointed states can be very time-consuming and inefficient.</p> |
| |
| <p>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.</p> |
| |
| <p>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.</p> |
| |
| <p>By changing the Spark configurations related to task scheduling, for example <code class="language-plaintext highlighter-rouge">spark.locality.wait</code>, 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.</p> |
| |
| <p>Specifically for built-in HDFS state store provider, users can check the state store metrics such as <code class="language-plaintext highlighter-rouge">loadedMapCacheHitCount</code> and <code class="language-plaintext highlighter-rouge">loadedMapCacheMissCount</code>. 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.</p> |
| |
| <h2 id="starting-streaming-queries">Starting Streaming Queries</h2> |
| <p>Once you have defined the final result DataFrame/Dataset, all that is left is for you to start the streaming computation. To do that, you have to use the <code class="language-plaintext highlighter-rouge">DataStreamWriter</code> |
| (<a href="api/scala/org/apache/spark/sql/streaming/DataStreamWriter.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/DataStreamWriter.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.DataStreamWriter.html#pyspark.sql.streaming.DataStreamWriter">Python</a> docs) |
| returned through <code class="language-plaintext highlighter-rouge">Dataset.writeStream()</code>. You will have to specify one or more of the following in this interface.</p> |
| |
| <ul> |
| <li> |
| <p><em>Details of the output sink:</em> Data format, location, etc.</p> |
| </li> |
| <li> |
| <p><em>Output mode:</em> Specify what gets written to the output sink.</p> |
| </li> |
| <li> |
| <p><em>Query name:</em> Optionally, specify a unique name of the query for identification.</p> |
| </li> |
| <li> |
| <p><em>Trigger interval:</em> 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.</p> |
| </li> |
| <li> |
| <p><em>Checkpoint location:</em> 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.</p> |
| </li> |
| </ul> |
| |
| <h4 id="output-modes">Output Modes</h4> |
| <p>There are a few types of output modes.</p> |
| |
| <ul> |
| <li> |
| <p><strong>Append mode (default)</strong> - 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 <code class="language-plaintext highlighter-rouge">select</code>, |
| <code class="language-plaintext highlighter-rouge">where</code>, <code class="language-plaintext highlighter-rouge">map</code>, <code class="language-plaintext highlighter-rouge">flatMap</code>, <code class="language-plaintext highlighter-rouge">filter</code>, <code class="language-plaintext highlighter-rouge">join</code>, etc. will support Append mode.</p> |
| </li> |
| <li> |
| <p><strong>Complete mode</strong> - The whole Result Table will be outputted to the sink after every trigger. |
| This is supported for aggregation queries.</p> |
| </li> |
| <li> |
| <p><strong>Update mode</strong> - (<em>Available since Spark 2.1.1</em>) 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.</p> |
| </li> |
| </ul> |
| |
| <p>Different types of streaming queries support different output modes. |
| Here is the compatibility matrix.</p> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Query Type</th> |
| <th></th> |
| <th>Supported Output Modes</th> |
| <th>Notes</th> |
| </tr> |
| </thead> |
| <tr> |
| <td rowspan="2" style="vertical-align: middle;">Queries with aggregation</td> |
| <td style="vertical-align: middle;">Aggregation on event-time with watermark</td> |
| <td style="vertical-align: middle;">Append, Update, Complete</td> |
| <td> |
| Append mode uses watermark to drop old aggregation state. But the output of a |
| windowed aggregation is delayed the late threshold specified in <code>withWatermark()</code> as by |
| the modes semantics, rows can be added to the Result Table only once after they are |
| finalized (i.e. after watermark is crossed). See the |
| <a href="#handling-late-data-and-watermarking">Late Data</a> section for more details. |
| <br /><br /> |
| Update mode uses watermark to drop old aggregation state. |
| <br /><br /> |
| Complete mode does not drop old aggregation state since by definition this mode |
| preserves all data in the Result Table. |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Other aggregations</td> |
| <td style="vertical-align: middle;">Complete, Update</td> |
| <td> |
| Since no watermark is defined (only defined in other category), |
| old aggregation state is not dropped. |
| <br /><br /> |
| Append mode is not supported as aggregates can update thus violating the semantics of |
| this mode. |
| </td> |
| </tr> |
| <tr> |
| <td colspan="2" style="vertical-align: middle;">Queries with <code>mapGroupsWithState</code></td> |
| <td style="vertical-align: middle;">Update</td> |
| <td style="vertical-align: middle;"> |
| Aggregations not allowed in a query with <code>mapGroupsWithState</code>. |
| </td> |
| </tr> |
| <tr> |
| <td rowspan="2" style="vertical-align: middle;">Queries with <code>flatMapGroupsWithState</code></td> |
| <td style="vertical-align: middle;">Append operation mode</td> |
| <td style="vertical-align: middle;">Append</td> |
| <td style="vertical-align: middle;"> |
| Aggregations are allowed after <code>flatMapGroupsWithState</code>. |
| </td> |
| </tr> |
| <tr> |
| <td style="vertical-align: middle;">Update operation mode</td> |
| <td style="vertical-align: middle;">Update</td> |
| <td style="vertical-align: middle;"> |
| Aggregations not allowed in a query with <code>flatMapGroupsWithState</code>. |
| </td> |
| </tr> |
| <tr> |
| <td colspan="2" style="vertical-align: middle;">Queries with <code>joins</code></td> |
| <td style="vertical-align: middle;">Append</td> |
| <td style="vertical-align: middle;"> |
| Update and Complete mode not supported yet. See the |
| <a href="#support-matrix-for-joins-in-streaming-queries">support matrix in the Join Operations section</a> |
| for more details on what types of joins are supported. |
| </td> |
| </tr> |
| <tr> |
| <td colspan="2" style="vertical-align: middle;">Other queries</td> |
| <td style="vertical-align: middle;">Append, Update</td> |
| <td style="vertical-align: middle;"> |
| Complete mode not supported as it is infeasible to keep all unaggregated data in the Result Table. |
| </td> |
| </tr> |
| <tr> |
| <td></td> |
| <td></td> |
| <td></td> |
| <td></td> |
| </tr> |
| </table> |
| |
| <h4 id="output-sinks">Output Sinks</h4> |
| <p>There are a few types of built-in output sinks.</p> |
| |
| <ul> |
| <li><strong>File sink</strong> - Stores the output to a directory.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="n">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> <span class="c1">// can be "orc", "json", "csv", etc.</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"path"</span><span class="o">,</span> <span class="s">"path/to/destination/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span></code></pre></figure> |
| |
| <ul> |
| <li><strong>Kafka sink</strong> - Stores the output to one or more topics in Kafka.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="n">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"topic"</span><span class="o">,</span> <span class="s">"updates"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span></code></pre></figure> |
| |
| <ul> |
| <li><strong>Foreach sink</strong> - Runs arbitrary computation on the records in the output. See later in the section for more details.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="n">writeStream</span> |
| <span class="o">.</span><span class="py">foreach</span><span class="o">(...)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span></code></pre></figure> |
| |
| <ul> |
| <li><strong>Console sink (for debugging)</strong> - 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.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="n">writeStream</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></code></pre></figure> |
| |
| <ul> |
| <li><strong>Memory sink (for debugging)</strong> - 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.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="n">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"memory"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">queryName</span><span class="o">(</span><span class="s">"tableName"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span></code></pre></figure> |
| |
| <p>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 |
| <a href="#fault-tolerance-semantics">fault-tolerance semantics</a>. |
| Here are the details of all the sinks in Spark.</p> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Sink</th> |
| <th>Supported Output Modes</th> |
| <th>Options</th> |
| <th>Fault-tolerant</th> |
| <th>Notes</th> |
| </tr> |
| </thead> |
| <tr> |
| <td><b>File Sink</b></td> |
| <td>Append</td> |
| <td> |
| <code>path</code>: path to the output directory, must be specified.<br /> |
| <code>retention</code>: time to live (TTL) for output files. Output files which batches were |
| committed older than TTL will be eventually excluded in metadata log. This means reader queries which read |
| the sink's output directory may not process them. You can provide the value as string format of the time. (like "12h", "7d", etc.) |
| By default it's disabled. |
| <br /><br /> |
| For file-format-specific options, see the related methods in DataFrameWriter |
| (<a href="api/scala/org/apache/spark/sql/DataFrameWriter.html">Scala</a>/<a href="api/java/org/apache/spark/sql/DataFrameWriter.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.DataStreamWriter.html#pyspark.sql.streaming.DataStreamWriter">Python</a>/<a href="api/R/write.stream.html">R</a>). |
| E.g. for "parquet" format options see <code>DataFrameWriter.parquet()</code> |
| </td> |
| <td>Yes (exactly-once)</td> |
| <td>Supports writes to partitioned tables. Partitioning by time may be useful.</td> |
| </tr> |
| <tr> |
| <td><b>Kafka Sink</b></td> |
| <td>Append, Update, Complete</td> |
| <td>See the <a href="structured-streaming-kafka-integration.html">Kafka Integration Guide</a></td> |
| <td>Yes (at-least-once)</td> |
| <td>More details in the <a href="structured-streaming-kafka-integration.html">Kafka Integration Guide</a></td> |
| </tr> |
| <tr> |
| <td><b>Foreach Sink</b></td> |
| <td>Append, Update, Complete</td> |
| <td>None</td> |
| <td>Yes (at-least-once)</td> |
| <td>More details in the <a href="#using-foreach-and-foreachbatch">next section</a></td> |
| </tr> |
| <tr> |
| <td><b>ForeachBatch Sink</b></td> |
| <td>Append, Update, Complete</td> |
| <td>None</td> |
| <td>Depends on the implementation</td> |
| <td>More details in the <a href="#using-foreach-and-foreachbatch">next section</a></td> |
| </tr> |
| |
| <tr> |
| <td><b>Console Sink</b></td> |
| <td>Append, Update, Complete</td> |
| <td> |
| <code>numRows</code>: Number of rows to print every trigger (default: 20) |
| <br /> |
| <code>truncate</code>: Whether to truncate the output if too long (default: true) |
| </td> |
| <td>No</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td><b>Memory Sink</b></td> |
| <td>Append, Complete</td> |
| <td>None</td> |
| <td>No. But in Complete Mode, restarted query will recreate the full table.</td> |
| <td>Table name is the query name.</td> |
| </tr> |
| <tr> |
| <td></td> |
| <td></td> |
| <td></td> |
| <td></td> |
| <td></td> |
| </tr> |
| </table> |
| |
| <p>Note that you have to call <code class="language-plaintext highlighter-rouge">start()</code> 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.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="c1"># ========== DF with no aggregations ========== |
| </span><span class="n">noAggDF</span> <span class="o">=</span> <span class="n">deviceDataDf</span><span class="p">.</span><span class="n">select</span><span class="p">(</span><span class="s">"device"</span><span class="p">).</span><span class="n">where</span><span class="p">(</span><span class="s">"signal > 10"</span><span class="p">)</span> |
| |
| <span class="c1"># Print new data to console |
| </span><span class="n">noAggDF</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># Write new data to Parquet files |
| </span><span class="n">noAggDF</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"parquet"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"checkpointLocation"</span><span class="p">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"path"</span><span class="p">,</span> <span class="s">"path/to/destination/dir"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># ========== DF with aggregation ========== |
| </span><span class="n">aggDF</span> <span class="o">=</span> <span class="n">df</span><span class="p">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s">"device"</span><span class="p">).</span><span class="n">count</span><span class="p">()</span> |
| |
| <span class="c1"># Print updated aggregations to console |
| </span><span class="n">aggDF</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="n">outputMode</span><span class="p">(</span><span class="s">"complete"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># Have all the aggregates in an in-memory table. The query name will be the table name |
| </span><span class="n">aggDF</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="n">queryName</span><span class="p">(</span><span class="s">"aggregates"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">outputMode</span><span class="p">(</span><span class="s">"complete"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"memory"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="n">spark</span><span class="p">.</span><span class="n">sql</span><span class="p">(</span><span class="s">"select * from aggregates"</span><span class="p">).</span><span class="n">show</span><span class="p">()</span> <span class="c1"># interactively query in-memory table</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// ========== DF with no aggregations ==========</span> |
| <span class="k">val</span> <span class="nv">noAggDF</span> <span class="k">=</span> <span class="nv">deviceDataDf</span><span class="o">.</span><span class="py">select</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="py">where</span><span class="o">(</span><span class="s">"signal > 10"</span><span class="o">)</span> |
| |
| <span class="c1">// Print new data to console</span> |
| <span class="n">noAggDF</span> |
| <span class="o">.</span><span class="py">writeStream</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="c1">// Write new data to Parquet files</span> |
| <span class="n">noAggDF</span> |
| <span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"path"</span><span class="o">,</span> <span class="s">"path/to/destination/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| |
| <span class="c1">// ========== DF with aggregation ==========</span> |
| <span class="k">val</span> <span class="nv">aggDF</span> <span class="k">=</span> <span class="nv">df</span><span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="py">count</span><span class="o">()</span> |
| |
| <span class="c1">// Print updated aggregations to console</span> |
| <span class="n">aggDF</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="c1">// Have all the aggregates in an in-memory table</span> |
| <span class="n">aggDF</span> |
| <span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">queryName</span><span class="o">(</span><span class="s">"aggregates"</span><span class="o">)</span> <span class="c1">// this query name will be the table name</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">"memory"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| |
| <span class="nv">spark</span><span class="o">.</span><span class="py">sql</span><span class="o">(</span><span class="s">"select * from aggregates"</span><span class="o">).</span><span class="py">show</span><span class="o">()</span> <span class="c1">// interactively query in-memory table</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// ========== DF with no aggregations ==========</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">noAggDF</span> <span class="o">=</span> <span class="n">deviceDataDf</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="na">where</span><span class="o">(</span><span class="s">"signal > 10"</span><span class="o">);</span> |
| |
| <span class="c1">// Print new data to console</span> |
| <span class="n">noAggDF</span> |
| <span class="o">.</span><span class="na">writeStream</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="c1">// Write new data to Parquet files</span> |
| <span class="n">noAggDF</span> |
| <span class="o">.</span><span class="na">writeStream</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"path"</span><span class="o">,</span> <span class="s">"path/to/destination/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span> |
| |
| <span class="c1">// ========== DF with aggregation ==========</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">aggDF</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"device"</span><span class="o">).</span><span class="na">count</span><span class="o">();</span> |
| |
| <span class="c1">// Print updated aggregations to console</span> |
| <span class="n">aggDF</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="c1">// Have all the aggregates in an in-memory table</span> |
| <span class="n">aggDF</span> |
| <span class="o">.</span><span class="na">writeStream</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">queryName</span><span class="o">(</span><span class="s">"aggregates"</span><span class="o">)</span> <span class="c1">// this query name will be the table name</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">"memory"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"select * from aggregates"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> <span class="c1">// interactively query in-memory table</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># ========== DF with no aggregations ==========</span><span class="w"> |
| </span><span class="n">noAggDF</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">select</span><span class="p">(</span><span class="n">where</span><span class="p">(</span><span class="n">deviceDataDf</span><span class="p">,</span><span class="w"> </span><span class="s2">"signal > 10"</span><span class="p">),</span><span class="w"> </span><span class="s2">"device"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Print new data to console</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">noAggDF</span><span class="p">,</span><span class="w"> </span><span class="s2">"console"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Write new data to Parquet files</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">noAggDF</span><span class="p">,</span><span class="w"> |
| </span><span class="s2">"parquet"</span><span class="p">,</span><span class="w"> |
| </span><span class="n">path</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"path/to/destination/dir"</span><span class="p">,</span><span class="w"> |
| </span><span class="n">checkpointLocation</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"path/to/checkpoint/dir"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># ========== DF with aggregation ==========</span><span class="w"> |
| </span><span class="n">aggDF</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">count</span><span class="p">(</span><span class="n">groupBy</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"device"</span><span class="p">))</span><span class="w"> |
| |
| </span><span class="c1"># Print updated aggregations to console</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">aggDF</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="c1"># Have all the aggregates in an in memory table. The query name will be the table name</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">aggDF</span><span class="p">,</span><span class="w"> </span><span class="s2">"memory"</span><span class="p">,</span><span class="w"> </span><span class="n">queryName</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"aggregates"</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="c1"># Interactively query in-memory table</span><span class="w"> |
| </span><span class="n">head</span><span class="p">(</span><span class="n">sql</span><span class="p">(</span><span class="s2">"select * from aggregates"</span><span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h5 id="using-foreach-and-foreachbatch">Using Foreach and ForeachBatch</h5> |
| <p>The <code class="language-plaintext highlighter-rouge">foreach</code> and <code class="language-plaintext highlighter-rouge">foreachBatch</code> operations allow you to apply arbitrary operations and writing |
| logic on the output of a streaming query. They have slightly different use cases - while <code class="language-plaintext highlighter-rouge">foreach</code> |
| allows custom write logic on every row, <code class="language-plaintext highlighter-rouge">foreachBatch</code> allows arbitrary operations |
| and custom logic on the output of each micro-batch. Let’s understand their usages in more detail.</p> |
| |
| <h6 id="foreachbatch">ForeachBatch</h6> |
| <p><code class="language-plaintext highlighter-rouge">foreachBatch(...)</code> 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.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="k">def</span> <span class="nf">foreach_batch_function</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">epoch_id</span><span class="p">):</span> |
| <span class="c1"># Transform and write batchDF |
| </span> <span class="k">pass</span> |
| |
| <span class="n">streamingDF</span><span class="p">.</span><span class="n">writeStream</span><span class="p">.</span><span class="n">foreachBatch</span><span class="p">(</span><span class="n">foreach_batch_function</span><span class="p">).</span><span class="n">start</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="nv">streamingDF</span><span class="o">.</span><span class="py">writeStream</span><span class="o">.</span><span class="py">foreachBatch</span> <span class="o">{</span> <span class="o">(</span><span class="n">batchDF</span><span class="k">:</span> <span class="kt">DataFrame</span><span class="o">,</span> <span class="n">batchId</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span> <span class="k">=></span> |
| <span class="c1">// Transform and write batchDF</span> |
| <span class="o">}.</span><span class="py">start</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="n">streamingDatasetOfString</span><span class="o">.</span><span class="na">writeStream</span><span class="o">().</span><span class="na">foreachBatch</span><span class="o">(</span> |
| <span class="k">new</span> <span class="nc">VoidFunction2</span><span class="o"><</span><span class="nc">Dataset</span><span class="o"><</span><span class="nc">String</span><span class="o">>,</span> <span class="nc">Long</span><span class="o">>()</span> <span class="o">{</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">call</span><span class="o">(</span><span class="nc">Dataset</span><span class="o"><</span><span class="nc">String</span><span class="o">></span> <span class="n">dataset</span><span class="o">,</span> <span class="nc">Long</span> <span class="n">batchId</span><span class="o">)</span> <span class="o">{</span> |
| <span class="c1">// Transform and write batchDF</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| <span class="o">).</span><span class="na">start</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| <p>R is not yet supported.</p> |
| </div> |
| |
| </div> |
| |
| <p>With <code class="language-plaintext highlighter-rouge">foreachBatch</code>, you can do the following.</p> |
| |
| <ul> |
| <li><strong>Reuse existing batch data sources</strong> - 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 <code class="language-plaintext highlighter-rouge">foreachBatch</code>, you can use the batch |
| data writers on the output of each micro-batch.</li> |
| <li><strong>Write to multiple locations</strong> - 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.</li> |
| </ul> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="nv">streamingDF</span><span class="o">.</span><span class="py">writeStream</span><span class="o">.</span><span class="py">foreachBatch</span> <span class="o">{</span> <span class="o">(</span><span class="n">batchDF</span><span class="k">:</span> <span class="kt">DataFrame</span><span class="o">,</span> <span class="n">batchId</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span> <span class="k">=></span> |
| <span class="nv">batchDF</span><span class="o">.</span><span class="py">persist</span><span class="o">()</span> |
| <span class="nv">batchDF</span><span class="o">.</span><span class="py">write</span><span class="o">.</span><span class="py">format</span><span class="o">(...).</span><span class="py">save</span><span class="o">(...)</span> <span class="c1">// location 1</span> |
| <span class="nv">batchDF</span><span class="o">.</span><span class="py">write</span><span class="o">.</span><span class="py">format</span><span class="o">(...).</span><span class="py">save</span><span class="o">(...)</span> <span class="c1">// location 2</span> |
| <span class="nv">batchDF</span><span class="o">.</span><span class="py">unpersist</span><span class="o">()</span> |
| <span class="o">}</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <ul> |
| <li><strong>Apply additional DataFrame operations</strong> - Many DataFrame and Dataset operations are not supported |
| in streaming DataFrames because Spark does not support generating incremental plans in those cases. |
| Using <code class="language-plaintext highlighter-rouge">foreachBatch</code>, 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.</li> |
| </ul> |
| |
| <p><strong>Note:</strong></p> |
| <ul> |
| <li>By default, <code class="language-plaintext highlighter-rouge">foreachBatch</code> 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.</li> |
| <li><code class="language-plaintext highlighter-rouge">foreachBatch</code> 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 <code class="language-plaintext highlighter-rouge">foreach</code> instead.</li> |
| </ul> |
| |
| <h6 id="foreach">Foreach</h6> |
| <p>If <code class="language-plaintext highlighter-rouge">foreachBatch</code> 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 <code class="language-plaintext highlighter-rouge">foreach</code>. |
| Specifically, you can express the data writing logic by dividing it into three methods: <code class="language-plaintext highlighter-rouge">open</code>, <code class="language-plaintext highlighter-rouge">process</code>, and <code class="language-plaintext highlighter-rouge">close</code>. |
| Since Spark 2.4, <code class="language-plaintext highlighter-rouge">foreach</code> is available in Scala, Java and Python.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <p>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.</p> |
| |
| <ul> |
| <li>First, the function takes a row as input.</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="k">def</span> <span class="nf">process_row</span><span class="p">(</span><span class="n">row</span><span class="p">):</span> |
| <span class="c1"># Write row to storage |
| </span> <span class="k">pass</span> |
| |
| <span class="n">query</span> <span class="o">=</span> <span class="n">streamingDF</span><span class="p">.</span><span class="n">writeStream</span><span class="p">.</span><span class="n">foreach</span><span class="p">(</span><span class="n">process_row</span><span class="p">).</span><span class="n">start</span><span class="p">()</span></code></pre></figure> |
| |
| <ul> |
| <li>Second, the object has a process method and optional open and close methods:</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="k">class</span> <span class="nc">ForeachWriter</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">open</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">partition_id</span><span class="p">,</span> <span class="n">epoch_id</span><span class="p">):</span> |
| <span class="c1"># Open connection. This method is optional in Python. |
| </span> <span class="k">pass</span> |
| |
| <span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">row</span><span class="p">):</span> |
| <span class="c1"># Write row to connection. This method is NOT optional in Python. |
| </span> <span class="k">pass</span> |
| |
| <span class="k">def</span> <span class="nf">close</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">error</span><span class="p">):</span> |
| <span class="c1"># Close the connection. This method in optional in Python. |
| </span> <span class="k">pass</span> |
| |
| <span class="n">query</span> <span class="o">=</span> <span class="n">streamingDF</span><span class="p">.</span><span class="n">writeStream</span><span class="p">.</span><span class="n">foreach</span><span class="p">(</span><span class="n">ForeachWriter</span><span class="p">()).</span><span class="n">start</span><span class="p">()</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <p>In Scala, you have to extend the class <code class="language-plaintext highlighter-rouge">ForeachWriter</code> (<a href="api/scala/org/apache/spark/sql/ForeachWriter.html">docs</a>).</p> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="nv">streamingDatasetOfString</span><span class="o">.</span><span class="py">writeStream</span><span class="o">.</span><span class="py">foreach</span><span class="o">(</span> |
| <span class="k">new</span> <span class="nc">ForeachWriter</span><span class="o">[</span><span class="kt">String</span><span class="o">]</span> <span class="o">{</span> |
| |
| <span class="k">def</span> <span class="nf">open</span><span class="o">(</span><span class="n">partitionId</span><span class="k">:</span> <span class="kt">Long</span><span class="o">,</span> <span class="n">version</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span><span class="k">:</span> <span class="kt">Boolean</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="c1">// Open connection</span> |
| <span class="o">}</span> |
| |
| <span class="k">def</span> <span class="nf">process</span><span class="o">(</span><span class="n">record</span><span class="k">:</span> <span class="kt">String</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="c1">// Write string to connection</span> |
| <span class="o">}</span> |
| |
| <span class="k">def</span> <span class="nf">close</span><span class="o">(</span><span class="n">errorOrNull</span><span class="k">:</span> <span class="kt">Throwable</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="c1">// Close the connection</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| <span class="o">).</span><span class="py">start</span><span class="o">()</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <p>In Java, you have to extend the class <code class="language-plaintext highlighter-rouge">ForeachWriter</code> (<a href="api/java/org/apache/spark/sql/ForeachWriter.html">docs</a>).</p> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">streamingDatasetOfString</span><span class="o">.</span><span class="na">writeStream</span><span class="o">().</span><span class="na">foreach</span><span class="o">(</span> |
| <span class="k">new</span> <span class="nc">ForeachWriter</span><span class="o"><</span><span class="nc">String</span><span class="o">>()</span> <span class="o">{</span> |
| |
| <span class="nd">@Override</span> <span class="kd">public</span> <span class="kt">boolean</span> <span class="nf">open</span><span class="o">(</span><span class="kt">long</span> <span class="n">partitionId</span><span class="o">,</span> <span class="kt">long</span> <span class="n">version</span><span class="o">)</span> <span class="o">{</span> |
| <span class="c1">// Open connection</span> |
| <span class="o">}</span> |
| |
| <span class="nd">@Override</span> <span class="kd">public</span> <span class="kt">void</span> <span class="nf">process</span><span class="o">(</span><span class="nc">String</span> <span class="n">record</span><span class="o">)</span> <span class="o">{</span> |
| <span class="c1">// Write string to connection</span> |
| <span class="o">}</span> |
| |
| <span class="nd">@Override</span> <span class="kd">public</span> <span class="kt">void</span> <span class="nf">close</span><span class="o">(</span><span class="nc">Throwable</span> <span class="n">errorOrNull</span><span class="o">)</span> <span class="o">{</span> |
| <span class="c1">// Close the connection</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| <span class="o">).</span><span class="na">start</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| <p>R is not yet supported.</p> |
| </div> |
| |
| </div> |
| |
| <p><strong>Execution semantics</strong> |
| When the streaming query is started, Spark calls the function or the object’s methods in the following way:</p> |
| |
| <ul> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>The lifecycle of the methods are as follows:</p> |
| |
| <ul> |
| <li> |
| <p>For each partition with partition_id:</p> |
| |
| <ul> |
| <li> |
| <p>For each batch/epoch of streaming data with epoch_id:</p> |
| |
| <ul> |
| <li> |
| <p>Method open(partitionId, epochId) is called.</p> |
| </li> |
| <li> |
| <p>If open(…) returns true, for each row in the partition and batch/epoch, method process(row) is called.</p> |
| </li> |
| <li> |
| <p>Method close(error) is called with error (if any) seen while processing rows.</p> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p><strong>Note:</strong> 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 <a href="https://issues.apache.org/jira/browse/SPARK-28650">SPARK-28650</a> for more details. |
| If you need deduplication on output, try out <code class="language-plaintext highlighter-rouge">foreachBatch</code> instead.</p> |
| </li> |
| </ul> |
| |
| <h4 id="streaming-table-apis">Streaming Table APIs</h4> |
| <p>Since Spark 3.1, you can also use <code class="language-plaintext highlighter-rouge">DataStreamReader.table()</code> to read tables as streaming DataFrames and use <code class="language-plaintext highlighter-rouge">DataStreamWriter.toTable()</code> to write streaming DataFrames as tables:</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># spark session |
| </span> |
| <span class="c1"># Create a streaming DataFrame |
| </span><span class="n">df</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="nb">format</span><span class="p">(</span><span class="s">"rate"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"rowsPerSecond"</span><span class="p">,</span> <span class="mi">10</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">load</span><span class="p">()</span> |
| |
| <span class="c1"># Write the streaming DataFrame to a table |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"checkpointLocation"</span><span class="p">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">toTable</span><span class="p">(</span><span class="s">"myTable"</span><span class="p">)</span> |
| |
| <span class="c1"># Check the table result |
| </span><span class="n">spark</span><span class="p">.</span><span class="n">read</span><span class="p">.</span><span class="n">table</span><span class="p">(</span><span class="s">"myTable"</span><span class="p">).</span><span class="n">show</span><span class="p">()</span> |
| |
| <span class="c1"># Transform the source dataset and write to a new table |
| </span><span class="n">spark</span><span class="p">.</span><span class="n">readStream</span> \ |
| <span class="p">.</span><span class="n">table</span><span class="p">(</span><span class="s">"myTable"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">select</span><span class="p">(</span><span class="s">"value"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"checkpointLocation"</span><span class="p">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"parquet"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">toTable</span><span class="p">(</span><span class="s">"newTable"</span><span class="p">)</span> |
| |
| <span class="c1"># Check the new table result |
| </span><span class="n">spark</span><span class="p">.</span><span class="n">read</span><span class="p">.</span><span class="n">table</span><span class="p">(</span><span class="s">"newTable"</span><span class="p">).</span><span class="n">show</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">val</span> <span class="nv">spark</span><span class="k">:</span> <span class="kt">SparkSession</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="c1">// Create a streaming DataFrame</span> |
| <span class="k">val</span> <span class="nv">df</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">"rate"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"rowsPerSecond"</span><span class="o">,</span> <span class="mi">10</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">load</span><span class="o">()</span> |
| |
| <span class="c1">// Write the streaming DataFrame to a table</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">toTable</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">)</span> |
| |
| <span class="c1">// Check the table result</span> |
| <span class="nv">spark</span><span class="o">.</span><span class="py">read</span><span class="o">.</span><span class="py">table</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">).</span><span class="py">show</span><span class="o">()</span> |
| |
| <span class="c1">// Transform the source dataset and write to a new table</span> |
| <span class="nv">spark</span><span class="o">.</span><span class="py">readStream</span> |
| <span class="o">.</span><span class="py">table</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">select</span><span class="o">(</span><span class="s">"value"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">toTable</span><span class="o">(</span><span class="s">"newTable"</span><span class="o">)</span> |
| |
| <span class="c1">// Check the new table result</span> |
| <span class="nv">spark</span><span class="o">.</span><span class="py">read</span><span class="o">.</span><span class="py">table</span><span class="o">(</span><span class="s">"newTable"</span><span class="o">).</span><span class="py">show</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="nc">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="c1">// Create a streaming DataFrame</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">df</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">"rate"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"rowsPerSecond"</span><span class="o">,</span> <span class="mi">10</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">load</span><span class="o">();</span> |
| |
| <span class="c1">// Write the streaming DataFrame to a table</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">toTable</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">);</span> |
| |
| <span class="c1">// Check the table result</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">table</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| |
| <span class="c1">// Transform the source dataset and write to a new table</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">table</span><span class="o">(</span><span class="s">"myTable"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"value"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">writeStream</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/checkpoint/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">toTable</span><span class="o">(</span><span class="s">"newTable"</span><span class="o">);</span> |
| |
| <span class="c1">// Check the new table result</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">table</span><span class="o">(</span><span class="s">"newTable"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| <p>Not available in R.</p> |
| </div> |
| |
| </div> |
| |
| <p>For more details, please check the docs for DataStreamReader (<a href="api/scala/org/apache/spark/sql/streaming/DataStreamReader.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/DataStreamReader.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.DataStreamReader.html#pyspark.sql.streaming.DataStreamReader">Python</a> docs) and DataStreamWriter (<a href="api/scala/org/apache/spark/sql/streaming/DataStreamWriter.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/DataStreamWriter.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.DataStreamWriter.html#pyspark.sql.streaming.DataStreamWriter">Python</a> docs).</p> |
| |
| <h4 id="triggers">Triggers</h4> |
| <p>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.</p> |
| |
| <table class="table table-striped"> |
| <thead> |
| <tr> |
| <th>Trigger Type</th> |
| <th>Description</th> |
| </tr> |
| </thead> |
| <tr> |
| <td><i>unspecified (default)</i></td> |
| <td> |
| 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. |
| </td> |
| </tr> |
| <tr> |
| <td><b>Fixed interval micro-batches</b></td> |
| <td> |
| The query will be executed with micro-batches mode, where micro-batches will be kicked off |
| at the user-specified intervals. |
| <ul> |
| <li>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.</li> |
| |
| <li>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).</li> |
| |
| <li>If no new data is available, then no micro-batch will be kicked off.</li> |
| </ul> |
| </td> |
| </tr> |
| <tr> |
| <td><b>One-time micro-batch</b><i>(deprecated)</i></td> |
| <td> |
| The query will execute <strong>only one</strong> micro-batch to process all the available data and then |
| stop on its own. This is 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 <b>Available-now micro-batch</b>, |
| as it provides the better guarantee of processing, fine-grained scale of batches, and better gradual processing |
| of watermark advancement including no-data batch. |
| </td> |
| </tr> |
| <tr> |
| <td><b>Available-now micro-batch</b></td> |
| <td> |
| Similar to queries one-time micro-batch trigger, the query will process all the available data and then |
| stop on its own. The difference is that, it will process the data in (possibly) multiple micro-batches |
| based on the source options (e.g. <code>maxFilesPerTrigger</code> for file source), which will result |
| in better query scalability. |
| <ul> |
| <li>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.</li> |
| |
| <li>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.</li> |
| </ul> |
| </td> |
| </tr> |
| <tr> |
| <td><b>Continuous with fixed checkpoint interval</b><br /><i>(experimental)</i></td> |
| <td> |
| The query will be executed in the new low-latency, continuous processing mode. Read more |
| about this in the <a href="#continuous-processing">Continuous Processing section</a> below. |
| </td> |
| </tr> |
| </table> |
| |
| <p>Here are a few code examples.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="c1"># Default trigger (runs micro-batch as soon as it can) |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># ProcessingTime trigger with two-seconds micro-batch interval |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">trigger</span><span class="p">(</span><span class="n">processingTime</span><span class="o">=</span><span class="s">'2 seconds'</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># One-time trigger (Deprecated, encouraged to use Available-now trigger) |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">trigger</span><span class="p">(</span><span class="n">once</span><span class="o">=</span><span class="bp">True</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># Available-now trigger |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">trigger</span><span class="p">(</span><span class="n">availableNow</span><span class="o">=</span><span class="bp">True</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="c1"># Continuous trigger with one-second checkpointing interval |
| </span><span class="n">df</span><span class="p">.</span><span class="n">writeStream</span> |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">)</span> |
| <span class="p">.</span><span class="n">trigger</span><span class="p">(</span><span class="n">continuous</span><span class="o">=</span><span class="s">'1 second'</span><span class="p">)</span> |
| <span class="p">.</span><span class="n">start</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.streaming.Trigger</span> |
| |
| <span class="c1">// Default trigger (runs micro-batch as soon as it can)</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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="c1">// ProcessingTime trigger with two-seconds micro-batch interval</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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">trigger</span><span class="o">(</span><span class="nv">Trigger</span><span class="o">.</span><span class="py">ProcessingTime</span><span class="o">(</span><span class="s">"2 seconds"</span><span class="o">))</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| |
| <span class="c1">// One-time trigger (Deprecated, encouraged to use Available-now trigger)</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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">trigger</span><span class="o">(</span><span class="nv">Trigger</span><span class="o">.</span><span class="py">Once</span><span class="o">())</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| |
| <span class="c1">// Available-now trigger</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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">trigger</span><span class="o">(</span><span class="nv">Trigger</span><span class="o">.</span><span class="py">AvailableNow</span><span class="o">())</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| |
| <span class="c1">// Continuous trigger with one-second checkpointing interval</span> |
| <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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">trigger</span><span class="o">(</span><span class="nv">Trigger</span><span class="o">.</span><span class="py">Continuous</span><span class="o">(</span><span class="s">"1 second"</span><span class="o">))</span> |
| <span class="o">.</span><span class="py">start</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="kn">import</span> <span class="nn">org.apache.spark.sql.streaming.Trigger</span> |
| |
| <span class="c1">// Default trigger (runs micro-batch as soon as it can)</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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="c1">// ProcessingTime trigger with two-seconds micro-batch interval</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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">trigger</span><span class="o">(</span><span class="nc">Trigger</span><span class="o">.</span><span class="na">ProcessingTime</span><span class="o">(</span><span class="s">"2 seconds"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span> |
| |
| <span class="c1">// One-time trigger (Deprecated, encouraged to use Available-now trigger)</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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">trigger</span><span class="o">(</span><span class="nc">Trigger</span><span class="o">.</span><span class="na">Once</span><span class="o">())</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span> |
| |
| <span class="c1">// Available-now trigger</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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">trigger</span><span class="o">(</span><span class="nc">Trigger</span><span class="o">.</span><span class="na">AvailableNow</span><span class="o">())</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span> |
| |
| <span class="c1">// Continuous trigger with one-second checkpointing interval</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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">trigger</span><span class="o">(</span><span class="nc">Trigger</span><span class="o">.</span><span class="na">Continuous</span><span class="o">(</span><span class="s">"1 second"</span><span class="o">))</span> |
| <span class="o">.</span><span class="na">start</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"># Default trigger (runs micro-batch as soon as it can)</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"console"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># ProcessingTime trigger with two-seconds micro-batch interval</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">df</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">trigger.processingTime</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"2 seconds"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># One-time trigger</span><span class="w"> |
| </span><span class="n">write.stream</span><span class="p">(</span><span class="n">df</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">trigger.once</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="kc">TRUE</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># Continuous trigger is not yet supported</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="managing-streaming-queries">Managing Streaming Queries</h2> |
| <p>The <code class="language-plaintext highlighter-rouge">StreamingQuery</code> object created when a query is started can be used to monitor and manage the query.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">query</span> <span class="o">=</span> <span class="n">df</span><span class="p">.</span><span class="n">writeStream</span><span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"console"</span><span class="p">).</span><span class="n">start</span><span class="p">()</span> <span class="c1"># get the query object |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="nb">id</span><span class="p">()</span> <span class="c1"># get the unique identifier of the running query that persists across restarts from checkpoint data |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">runId</span><span class="p">()</span> <span class="c1"># get the unique id of this run of the query, which will be generated at every start/restart |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">name</span><span class="p">()</span> <span class="c1"># get the name of the auto-generated or user-specified name |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">explain</span><span class="p">()</span> <span class="c1"># print detailed explanations of the query |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">stop</span><span class="p">()</span> <span class="c1"># stop the query |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">awaitTermination</span><span class="p">()</span> <span class="c1"># block until query is terminated, with stop() or with error |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">exception</span><span class="p">()</span> <span class="c1"># the exception if the query has been terminated with error |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">recentProgress</span> <span class="c1"># a list of the most recent progress updates for this query |
| </span> |
| <span class="n">query</span><span class="p">.</span><span class="n">lastProgress</span> <span class="c1"># the most recent progress update of this streaming query</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">query</span> <span class="k">=</span> <span class="nv">df</span><span class="o">.</span><span class="py">writeStream</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="py">start</span><span class="o">()</span> <span class="c1">// get the query object</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">id</span> <span class="c1">// get the unique identifier of the running query that persists across restarts from checkpoint data</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">runId</span> <span class="c1">// get the unique id of this run of the query, which will be generated at every start/restart</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">name</span> <span class="c1">// get the name of the auto-generated or user-specified name</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">explain</span><span class="o">()</span> <span class="c1">// print detailed explanations of the query</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">stop</span><span class="o">()</span> <span class="c1">// stop the query</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">awaitTermination</span><span class="o">()</span> <span class="c1">// block until query is terminated, with stop() or with error</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">exception</span> <span class="c1">// the exception if the query has been terminated with error</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">recentProgress</span> <span class="c1">// an array of the most recent progress updates for this query</span> |
| |
| <span class="nv">query</span><span class="o">.</span><span class="py">lastProgress</span> <span class="c1">// the most recent progress update of this streaming query</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="nc">StreamingQuery</span> <span class="n">query</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="na">writeStream</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="na">start</span><span class="o">();</span> <span class="c1">// get the query object</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">id</span><span class="o">();</span> <span class="c1">// get the unique identifier of the running query that persists across restarts from checkpoint data</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">runId</span><span class="o">();</span> <span class="c1">// get the unique id of this run of the query, which will be generated at every start/restart</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">name</span><span class="o">();</span> <span class="c1">// get the name of the auto-generated or user-specified name</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">explain</span><span class="o">();</span> <span class="c1">// print detailed explanations of the query</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">stop</span><span class="o">();</span> <span class="c1">// stop the query</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">awaitTermination</span><span class="o">();</span> <span class="c1">// block until query is terminated, with stop() or with error</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">exception</span><span class="o">();</span> <span class="c1">// the exception if the query has been terminated with error</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">recentProgress</span><span class="o">();</span> <span class="c1">// an array of the most recent progress updates for this query</span> |
| |
| <span class="n">query</span><span class="o">.</span><span class="na">lastProgress</span><span class="o">();</span> <span class="c1">// the most recent progress update of this streaming query</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">query</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">write.stream</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"console"</span><span class="p">)</span><span class="w"> </span><span class="c1"># get the query object</span><span class="w"> |
| |
| </span><span class="n">queryName</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> </span><span class="c1"># get the name of the auto-generated or user-specified name</span><span class="w"> |
| |
| </span><span class="n">explain</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> </span><span class="c1"># print detailed explanations of the query</span><span class="w"> |
| |
| </span><span class="n">stopQuery</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> </span><span class="c1"># stop the query</span><span class="w"> |
| |
| </span><span class="n">awaitTermination</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> </span><span class="c1"># block until query is terminated, with stop() or with error</span><span class="w"> |
| |
| </span><span class="n">lastProgress</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> </span><span class="c1"># the most recent progress update of this streaming query</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use <code class="language-plaintext highlighter-rouge">sparkSession.streams()</code> to get the <code class="language-plaintext highlighter-rouge">StreamingQueryManager</code> |
| (<a href="api/scala/org/apache/spark/sql/streaming/StreamingQueryManager.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryManager.html#pyspark.sql.streaming.StreamingQueryManager">Python</a> docs) |
| that can be used to manage the currently active queries.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># spark session |
| </span> |
| <span class="n">spark</span><span class="p">.</span><span class="n">streams</span><span class="p">.</span><span class="n">active</span> <span class="c1"># get the list of currently active streaming queries |
| </span> |
| <span class="n">spark</span><span class="p">.</span><span class="n">streams</span><span class="p">.</span><span class="n">get</span><span class="p">(</span><span class="nb">id</span><span class="p">)</span> <span class="c1"># get a query object by its unique id |
| </span> |
| <span class="n">spark</span><span class="p">.</span><span class="n">streams</span><span class="p">.</span><span class="n">awaitAnyTermination</span><span class="p">()</span> <span class="c1"># block until any one of them terminates</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">spark</span><span class="k">:</span> <span class="kt">SparkSession</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="nv">spark</span><span class="o">.</span><span class="py">streams</span><span class="o">.</span><span class="py">active</span> <span class="c1">// get the list of currently active streaming queries</span> |
| |
| <span class="nv">spark</span><span class="o">.</span><span class="py">streams</span><span class="o">.</span><span class="py">get</span><span class="o">(</span><span class="n">id</span><span class="o">)</span> <span class="c1">// get a query object by its unique id</span> |
| |
| <span class="nv">spark</span><span class="o">.</span><span class="py">streams</span><span class="o">.</span><span class="py">awaitAnyTermination</span><span class="o">()</span> <span class="c1">// block until any one of them terminates</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="nc">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">streams</span><span class="o">().</span><span class="na">active</span><span class="o">();</span> <span class="c1">// get the list of currently active streaming queries</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">streams</span><span class="o">().</span><span class="na">get</span><span class="o">(</span><span class="n">id</span><span class="o">);</span> <span class="c1">// get a query object by its unique id</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">streams</span><span class="o">().</span><span class="na">awaitAnyTermination</span><span class="o">();</span> <span class="c1">// block until any one of them terminates</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash">Not available <span class="k">in </span>R.</code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="monitoring-streaming-queries">Monitoring Streaming Queries</h2> |
| <p>There are multiple ways to monitor active streaming queries. You can either push metrics to external systems using Spark’s Dropwizard Metrics support, or access them programmatically.</p> |
| |
| <h3 id="reading-metrics-interactively">Reading Metrics Interactively</h3> |
| |
| <p>You can directly get the current status and metrics of an active query using |
| <code class="language-plaintext highlighter-rouge">streamingQuery.lastProgress()</code> and <code class="language-plaintext highlighter-rouge">streamingQuery.status()</code>. |
| <code class="language-plaintext highlighter-rouge">lastProgress()</code> returns a <code class="language-plaintext highlighter-rouge">StreamingQueryProgress</code> object |
| in <a href="api/scala/org/apache/spark/sql/streaming/StreamingQueryProgress.html">Scala</a> |
| and <a href="api/java/org/apache/spark/sql/streaming/StreamingQueryProgress.html">Java</a> |
| and a dictionary with the same fields in Python. It has all the information about |
| the progress made in the last trigger of the stream - what data was processed, |
| what were the processing rates, latencies, etc. There is also |
| <code class="language-plaintext highlighter-rouge">streamingQuery.recentProgress</code> which returns an array of last few progresses.</p> |
| |
| <p>In addition, <code class="language-plaintext highlighter-rouge">streamingQuery.status()</code> returns a <code class="language-plaintext highlighter-rouge">StreamingQueryStatus</code> object |
| in <a href="api/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.html">Scala</a> |
| and <a href="api/java/org/apache/spark/sql/streaming/StreamingQueryStatus.html">Java</a> |
| 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.</p> |
| |
| <p>Here are a few examples.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">query</span> <span class="o">=</span> <span class="p">...</span> <span class="c1"># a StreamingQuery |
| </span><span class="k">print</span><span class="p">(</span><span class="n">query</span><span class="p">.</span><span class="n">lastProgress</span><span class="p">)</span> |
| |
| <span class="s">''' |
| Will print something like the following. |
| |
| {u'stateOperators': [], u'eventTime': {u'watermark': u'2016-12-14T18:45:24.873Z'}, u'name': u'MyQuery', u'timestamp': u'2016-12-14T18:45:24.873Z', u'processedRowsPerSecond': 200.0, u'inputRowsPerSecond': 120.0, u'numInputRows': 10, u'sources': [{u'description': u'KafkaSource[Subscribe[topic-0]]', u'endOffset': {u'topic-0': {u'1': 134, u'0': 534, u'3': 21, u'2': 0, u'4': 115}}, u'processedRowsPerSecond': 200.0, u'inputRowsPerSecond': 120.0, u'numInputRows': 10, u'startOffset': {u'topic-0': {u'1': 1, u'0': 1, u'3': 1, u'2': 0, u'4': 1}}}], u'durationMs': {u'getOffset': 2, u'triggerExecution': 3}, u'runId': u'88e2ff94-ede0-45a8-b687-6316fbef529a', u'id': u'ce011fdc-8762-4dcb-84eb-a77333e28109', u'sink': {u'description': u'MemorySink'}} |
| '''</span> |
| |
| <span class="k">print</span><span class="p">(</span><span class="n">query</span><span class="p">.</span><span class="n">status</span><span class="p">)</span> |
| <span class="s">''' |
| Will print something like the following. |
| |
| {u'message': u'Waiting for data to arrive', u'isTriggerActive': False, u'isDataAvailable': False} |
| '''</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">query</span><span class="k">:</span> <span class="kt">StreamingQuery</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="nf">println</span><span class="o">(</span><span class="nv">query</span><span class="o">.</span><span class="py">lastProgress</span><span class="o">)</span> |
| |
| <span class="cm">/* Will print something like the following. |
| |
| { |
| "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" |
| } |
| } |
| */</span> |
| |
| |
| <span class="nf">println</span><span class="o">(</span><span class="nv">query</span><span class="o">.</span><span class="py">status</span><span class="o">)</span> |
| |
| <span class="cm">/* Will print something like the following. |
| { |
| "message" : "Waiting for data to arrive", |
| "isDataAvailable" : false, |
| "isTriggerActive" : false |
| } |
| */</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span class="nc">StreamingQuery</span> <span class="n">query</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="nc">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="n">query</span><span class="o">.</span><span class="na">lastProgress</span><span class="o">());</span> |
| <span class="cm">/* Will print something like the following. |
| |
| { |
| "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" |
| } |
| } |
| */</span> |
| |
| |
| <span class="nc">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="n">query</span><span class="o">.</span><span class="na">status</span><span class="o">());</span> |
| <span class="cm">/* Will print something like the following. |
| { |
| "message" : "Waiting for data to arrive", |
| "isDataAvailable" : false, |
| "isTriggerActive" : false |
| } |
| */</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">query</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">...</span><span class="w"> </span><span class="c1"># a StreamingQuery</span><span class="w"> |
| </span><span class="n">lastProgress</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="s1">''' |
| Will print something like the following. |
| |
| { |
| "id" : "8c57e1ec-94b5-4c99-b100-f694162df0b9", |
| "runId" : "ae505c5a-a64e-4896-8c28-c7cbaf926f16", |
| "name" : null, |
| "timestamp" : "2017-04-26T08:27:28.835Z", |
| "numInputRows" : 0, |
| "inputRowsPerSecond" : 0.0, |
| "processedRowsPerSecond" : 0.0, |
| "durationMs" : { |
| "getOffset" : 0, |
| "triggerExecution" : 1 |
| }, |
| "stateOperators" : [ { |
| "numRowsTotal" : 4, |
| "numRowsUpdated" : 0 |
| } ], |
| "sources" : [ { |
| "description" : "TextSocketSource[host: localhost, port: 9999]", |
| "startOffset" : 1, |
| "endOffset" : 1, |
| "numInputRows" : 0, |
| "inputRowsPerSecond" : 0.0, |
| "processedRowsPerSecond" : 0.0 |
| } ], |
| "sink" : { |
| "description" : "org.apache.spark.sql.execution.streaming.ConsoleSink@76b37531" |
| } |
| } |
| '''</span><span class="w"> |
| |
| </span><span class="n">status</span><span class="p">(</span><span class="n">query</span><span class="p">)</span><span class="w"> |
| </span><span class="s1">''' |
| Will print something like the following. |
| |
| { |
| "message" : "Waiting for data to arrive", |
| "isDataAvailable" : false, |
| "isTriggerActive" : false |
| } |
| '''</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="reporting-metrics-programmatically-using-asynchronous-apis">Reporting Metrics programmatically using Asynchronous APIs</h3> |
| |
| <p>You can also asynchronously monitor all queries associated with a |
| <code class="language-plaintext highlighter-rouge">SparkSession</code> by attaching a <code class="language-plaintext highlighter-rouge">StreamingQueryListener</code> |
| (<a href="api/scala/org/apache/spark/sql/streaming/StreamingQueryListener.html">Scala</a>/<a href="api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html">Java</a>/<a href="api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryListener.html">Python</a> docs). |
| Once you attach your custom <code class="language-plaintext highlighter-rouge">StreamingQueryListener</code> object with |
| <code class="language-plaintext highlighter-rouge">sparkSession.streams.addListener()</code>, you will get callbacks when a query is started and |
| stopped and when there is progress made in an active query. Here is an example,</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span> <span class="o">=</span> <span class="p">...</span> |
| |
| <span class="k">class</span> <span class="nc">Listener</span><span class="p">(</span><span class="n">StreamingQueryListener</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">onQueryStarted</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">event</span><span class="p">):</span> |
| <span class="k">print</span><span class="p">(</span><span class="s">"Query started: "</span> <span class="o">+</span> <span class="n">queryStarted</span><span class="p">.</span><span class="nb">id</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">onQueryProgress</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">event</span><span class="p">):</span> |
| <span class="n">println</span><span class="p">(</span><span class="s">"Query terminated: "</span> <span class="o">+</span> <span class="n">queryTerminated</span><span class="p">.</span><span class="nb">id</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">onQueryTerminated</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">event</span><span class="p">):</span> |
| <span class="n">println</span><span class="p">(</span><span class="s">"Query made progress: "</span> <span class="o">+</span> <span class="n">queryProgress</span><span class="p">.</span><span class="n">progress</span><span class="p">)</span> |
| |
| |
| <span class="n">spark</span><span class="p">.</span><span class="n">streams</span><span class="p">.</span><span class="n">addListener</span><span class="p">(</span><span class="n">Listener</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">val</span> <span class="nv">spark</span><span class="k">:</span> <span class="kt">SparkSession</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="nv">spark</span><span class="o">.</span><span class="py">streams</span><span class="o">.</span><span class="py">addListener</span><span class="o">(</span><span class="k">new</span> <span class="nc">StreamingQueryListener</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">override</span> <span class="k">def</span> <span class="nf">onQueryStarted</span><span class="o">(</span><span class="n">queryStarted</span><span class="k">:</span> <span class="kt">QueryStartedEvent</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="nf">println</span><span class="o">(</span><span class="s">"Query started: "</span> <span class="o">+</span> <span class="nv">queryStarted</span><span class="o">.</span><span class="py">id</span><span class="o">)</span> |
| <span class="o">}</span> |
| <span class="k">override</span> <span class="k">def</span> <span class="nf">onQueryTerminated</span><span class="o">(</span><span class="n">queryTerminated</span><span class="k">:</span> <span class="kt">QueryTerminatedEvent</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="nf">println</span><span class="o">(</span><span class="s">"Query terminated: "</span> <span class="o">+</span> <span class="nv">queryTerminated</span><span class="o">.</span><span class="py">id</span><span class="o">)</span> |
| <span class="o">}</span> |
| <span class="k">override</span> <span class="k">def</span> <span class="nf">onQueryProgress</span><span class="o">(</span><span class="n">queryProgress</span><span class="k">:</span> <span class="kt">QueryProgressEvent</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="nf">println</span><span class="o">(</span><span class="s">"Query made progress: "</span> <span class="o">+</span> <span class="nv">queryProgress</span><span class="o">.</span><span class="py">progress</span><span class="o">)</span> |
| <span class="o">}</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="nc">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="o">...</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">streams</span><span class="o">().</span><span class="na">addListener</span><span class="o">(</span><span class="k">new</span> <span class="nc">StreamingQueryListener</span><span class="o">()</span> <span class="o">{</span> |
| <span class="nd">@Override</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">onQueryStarted</span><span class="o">(</span><span class="nc">QueryStartedEvent</span> <span class="n">queryStarted</span><span class="o">)</span> <span class="o">{</span> |
| <span class="nc">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"Query started: "</span> <span class="o">+</span> <span class="n">queryStarted</span><span class="o">.</span><span class="na">id</span><span class="o">());</span> |
| <span class="o">}</span> |
| <span class="nd">@Override</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">onQueryTerminated</span><span class="o">(</span><span class="nc">QueryTerminatedEvent</span> <span class="n">queryTerminated</span><span class="o">)</span> <span class="o">{</span> |
| <span class="nc">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"Query terminated: "</span> <span class="o">+</span> <span class="n">queryTerminated</span><span class="o">.</span><span class="na">id</span><span class="o">());</span> |
| <span class="o">}</span> |
| <span class="nd">@Override</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">onQueryProgress</span><span class="o">(</span><span class="nc">QueryProgressEvent</span> <span class="n">queryProgress</span><span class="o">)</span> <span class="o">{</span> |
| <span class="nc">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"Query made progress: "</span> <span class="o">+</span> <span class="n">queryProgress</span><span class="o">.</span><span class="na">progress</span><span class="o">());</span> |
| <span class="o">}</span> |
| <span class="o">});</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash">Not available <span class="k">in </span>R.</code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="reporting-metrics-using-dropwizard">Reporting Metrics using Dropwizard</h3> |
| <p>Spark supports reporting metrics using the <a href="monitoring.html#metrics">Dropwizard Library</a>. To enable metrics of Structured Streaming queries to be reported as well, you have to explicitly enable the configuration <code class="language-plaintext highlighter-rouge">spark.sql.streaming.metricsEnabled</code> in the SparkSession.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span><span class="p">.</span><span class="n">conf</span><span class="p">.</span><span class="nb">set</span><span class="p">(</span><span class="s">"spark.sql.streaming.metricsEnabled"</span><span class="p">,</span> <span class="s">"true"</span><span class="p">)</span> |
| <span class="c1"># or |
| </span><span class="n">spark</span><span class="p">.</span><span class="n">sql</span><span class="p">(</span><span class="s">"SET spark.sql.streaming.metricsEnabled=true"</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="nv">spark</span><span class="o">.</span><span class="py">conf</span><span class="o">.</span><span class="py">set</span><span class="o">(</span><span class="s">"spark.sql.streaming.metricsEnabled"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="c1">// or</span> |
| <span class="nv">spark</span><span class="o">.</span><span class="py">sql</span><span class="o">(</span><span class="s">"SET spark.sql.streaming.metricsEnabled=true"</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="n">spark</span><span class="o">.</span><span class="na">conf</span><span class="o">().</span><span class="na">set</span><span class="o">(</span><span class="s">"spark.sql.streaming.metricsEnabled"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">);</span> |
| <span class="c1">// or</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SET spark.sql.streaming.metricsEnabled=true"</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">sql</span><span class="p">(</span><span class="s2">"SET spark.sql.streaming.metricsEnabled=true"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>All queries started in the SparkSession after this configuration has been enabled will report metrics through Dropwizard to whatever <a href="monitoring.html#metrics">sinks</a> have been configured (e.g. Ganglia, Graphite, JMX, etc.).</p> |
| |
| <h2 id="recovering-from-failures-with-checkpointing">Recovering from Failures with Checkpointing</h2> |
| <p>In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write-ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger) and the running aggregates (e.g. word counts in the <a href="#quick-example">quick example</a>) to the checkpoint location. This checkpoint location has to be a path in an HDFS compatible file system, and can be set as an option in the DataStreamWriter when <a href="#starting-streaming-queries">starting a query</a>.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">aggDF</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="n">outputMode</span><span class="p">(</span><span class="s">"complete"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"checkpointLocation"</span><span class="p">,</span> <span class="s">"path/to/HDFS/dir"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"memory"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">start</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="n">aggDF</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">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/HDFS/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"memory"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</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="n">aggDF</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">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"path/to/HDFS/dir"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"memory"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">start</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">write.stream</span><span class="p">(</span><span class="n">aggDF</span><span class="p">,</span><span class="w"> </span><span class="s2">"memory"</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">checkpointLocation</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"path/to/HDFS/dir"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="recovery-semantics-after-changes-in-a-streaming-query">Recovery Semantics after Changes in a Streaming Query</h2> |
| <p>There are limitations on what changes in a streaming query are allowed between restarts from the |
| same checkpoint location. Here are a few kinds of changes that are either not allowed, or |
| the effect of the change is not well-defined. For all of them:</p> |
| |
| <ul> |
| <li> |
| <p>The term <em>allowed</em> means you can do the specified change but whether the semantics of its effect |
| is well-defined depends on the query and the change.</p> |
| </li> |
| <li> |
| <p>The term <em>not allowed</em> means you should not do the specified change as the restarted query is likely |
| to fail with unpredictable errors. <code class="language-plaintext highlighter-rouge">sdf</code> represents a streaming DataFrame/Dataset |
| generated with sparkSession.readStream.</p> |
| </li> |
| </ul> |
| |
| <p><strong>Types of changes</strong></p> |
| |
| <ul> |
| <li> |
| <p><em>Changes in the number or type (i.e. different source) of input sources</em>: This is not allowed.</p> |
| </li> |
| <li> |
| <p><em>Changes in the parameters of input sources</em>: 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.</p> |
| |
| <ul> |
| <li> |
| <p>Addition/deletion/modification of rate limits is allowed: <code class="language-plaintext highlighter-rouge">spark.readStream.format("kafka").option("subscribe", "topic")</code> to <code class="language-plaintext highlighter-rouge">spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)</code></p> |
| </li> |
| <li> |
| <p>Changes to subscribed topics/files are generally not allowed as the results are unpredictable: <code class="language-plaintext highlighter-rouge">spark.readStream.format("kafka").option("subscribe", "topic")</code> to <code class="language-plaintext highlighter-rouge">spark.readStream.format("kafka").option("subscribe", "newTopic")</code></p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p><em>Changes in the type of output sink</em>: 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.</p> |
| |
| <ul> |
| <li> |
| <p>File sink to Kafka sink is allowed. Kafka will see only the new data.</p> |
| </li> |
| <li> |
| <p>Kafka sink to file sink is not allowed.</p> |
| </li> |
| <li> |
| <p>Kafka sink changed to foreach, or vice versa is allowed.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p><em>Changes in the parameters of output sink</em>: 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.</p> |
| |
| <ul> |
| <li> |
| <p>Changes to output directory of a file sink are not allowed: <code class="language-plaintext highlighter-rouge">sdf.writeStream.format("parquet").option("path", "/somePath")</code> to <code class="language-plaintext highlighter-rouge">sdf.writeStream.format("parquet").option("path", "/anotherPath")</code></p> |
| </li> |
| <li> |
| <p>Changes to output topic are allowed: <code class="language-plaintext highlighter-rouge">sdf.writeStream.format("kafka").option("topic", "someTopic")</code> to <code class="language-plaintext highlighter-rouge">sdf.writeStream.format("kafka").option("topic", "anotherTopic")</code></p> |
| </li> |
| <li> |
| <p>Changes to the user-defined foreach sink (that is, the <code class="language-plaintext highlighter-rouge">ForeachWriter</code> code) are allowed, but the semantics of the change depends on the code.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p><em>Changes in projection / filter / map-like operations</em>: Some cases are allowed. For example:</p> |
| |
| <ul> |
| <li> |
| <p>Addition / deletion of filters is allowed: <code class="language-plaintext highlighter-rouge">sdf.selectExpr("a")</code> to <code class="language-plaintext highlighter-rouge">sdf.where(...).selectExpr("a").filter(...)</code>.</p> |
| </li> |
| <li> |
| <p>Changes in projections with same output schema are allowed: <code class="language-plaintext highlighter-rouge">sdf.selectExpr("stringColumn AS json").writeStream</code> to <code class="language-plaintext highlighter-rouge">sdf.selectExpr("anotherStringColumn AS json").writeStream</code></p> |
| </li> |
| <li> |
| <p>Changes in projections with different output schema are conditionally allowed: <code class="language-plaintext highlighter-rouge">sdf.selectExpr("a").writeStream</code> to <code class="language-plaintext highlighter-rouge">sdf.selectExpr("b").writeStream</code> is allowed only if the output sink allows the schema change from <code class="language-plaintext highlighter-rouge">"a"</code> to <code class="language-plaintext highlighter-rouge">"b"</code>.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p><em>Changes in stateful operations</em>: 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 |
| <em>any changes (that is, additions, deletions, or schema modifications) to the stateful operations of a streaming query are not allowed between restarts</em>. |
| Here is the list of stateful operations whose schema should not be changed between restarts in order to ensure state recovery:</p> |
| |
| <ul> |
| <li> |
| <p><em>Streaming aggregation</em>: For example, <code class="language-plaintext highlighter-rouge">sdf.groupBy("a").agg(...)</code>. Any change in number or type of grouping keys or aggregates is not allowed.</p> |
| </li> |
| <li> |
| <p><em>Streaming deduplication</em>: For example, <code class="language-plaintext highlighter-rouge">sdf.dropDuplicates("a")</code>. Any change in number or type of deduplicating columns is not allowed.</p> |
| </li> |
| <li> |
| <p><em>Stream-stream join</em>: For example, <code class="language-plaintext highlighter-rouge">sdf1.join(sdf2, ...)</code> (i.e. both inputs are generated with <code class="language-plaintext highlighter-rouge">sparkSession.readStream</code>). 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.</p> |
| </li> |
| <li> |
| <p><em>Arbitrary stateful operation</em>: For example, <code class="language-plaintext highlighter-rouge">sdf.groupByKey(...).mapGroupsWithState(...)</code> or <code class="language-plaintext highlighter-rouge">sdf.groupByKey(...).flatMapGroupsWithState(...)</code>. |
| Any change to the schema of the user-defined state and the type of timeout is not allowed. |
| Any change within the user-defined state-mapping function are allowed, but the semantic effect of the change depends on the user-defined logic. |
| If you really want to support state schema changes, then you can explicitly encode/decode your complex state data |
| structures into bytes using an encoding/decoding scheme that supports schema migration. For example, |
| if you save your state as Avro-encoded bytes, then you are free to change the Avro-state-schema between query |
| restarts as the binary state will always be restored successfully.</p> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h1 id="asynchronous-progress-tracking">Asynchronous Progress Tracking</h1> |
| <h2 id="what-is-it">What is it?</h2> |
| |
| <p>Asynchronous progress tracking allows streaming queries to checkpoint progress asynchronously and in parallel to the actual data processing within a micro-batch, reducing latency associated with maintaining the offset log and commit log.</p> |
| |
| <p><img src="img/async-progress.png" alt="Async Progress Tracking" /></p> |
| |
| <h2 id="how-does-it-work">How does it work?</h2> |
| |
| <p>Structured Streaming relies on persisting and managing offsets as progress indicators for query processing. Offset management operation directly impacts processing latency, because no data processing can occur until these operations are complete. Asynchronous progress tracking enables streaming queries to checkpoint progress without being impacted by these offset management operations.</p> |
| |
| <h2 id="how-to-use-it">How to use it?</h2> |
| |
| <p>The code snippet below provides an example of how to use this feature:</p> |
| <div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">stream</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">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"subscribe"</span><span class="o">,</span> <span class="s">"in"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">load</span><span class="o">()</span> |
| <span class="k">val</span> <span class="nv">query</span> <span class="k">=</span> <span class="nv">stream</span><span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"topic"</span><span class="o">,</span> <span class="s">"out"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"checkpointLocation"</span><span class="o">,</span> <span class="s">"/tmp/checkpoint"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"asyncProgressTrackingEnabled"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">start</span><span class="o">()</span> |
| </code></pre></div></div> |
| |
| <p>The table below describes the configurations for this feature and default values associated with them.</p> |
| |
| <table> |
| <thead> |
| <tr> |
| <th>Option</th> |
| <th>Value</th> |
| <th>Default</th> |
| <th>Description</th> |
| </tr> |
| </thead> |
| <tbody> |
| <tr> |
| <td>asyncProgressTrackingEnabled</td> |
| <td>true/false</td> |
| <td>false</td> |
| <td>enable or disable asynchronous progress tracking</td> |
| </tr> |
| <tr> |
| <td>asyncProgressTrackingCheckpointIntervalMs</td> |
| <td>millisecond</td> |
| <td>1000</td> |
| <td>the interval in which we commit offsets and completion commits</td> |
| </tr> |
| </tbody> |
| </table> |
| |
| <h2 id="limitations">Limitations</h2> |
| <p>The initial version of the feature has the following limitations:</p> |
| |
| <ul> |
| <li>Asynchronous progress tracking is only supported in stateless queries using Kafka Sink</li> |
| <li>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.</li> |
| </ul> |
| |
| <h2 id="switching-the-setting-off">Switching the setting off</h2> |
| <p>Turning the async progress tracking off may cause the following exception to be thrown</p> |
| |
| <div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nv">java</span><span class="o">.</span><span class="py">lang</span><span class="o">.</span><span class="py">IllegalStateException</span><span class="k">:</span> <span class="kt">batch</span> <span class="kt">x</span> <span class="kt">doesn't</span> <span class="kt">exist</span> |
| </code></pre></div></div> |
| |
| <p>Also the following error message may be printed in the driver logs:</p> |
| |
| <div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>The offset log for batch x doesn't exist, which is required to restart the query from the latest batch x from the offset log. Please ensure there are two subsequent offset logs available for the latest batch via manually deleting the offset file(s). Please also ensure the latest batch for commit log is equal or one batch earlier than the latest batch for offset log. |
| </code></pre></div></div> |
| |
| <p>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 “asyncProgressTrackingCheckpointIntervalMs” 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.</p> |
| |
| <h1 id="continuous-processing">Continuous Processing</h1> |
| <h2 class="no_toc" id="experimental">[Experimental]</h2> |
| |
| <p><strong>Continuous processing</strong> 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 <em>micro-batch processing</em> 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).</p> |
| |
| <p>To run a supported query in continuous processing mode, all you need to do is specify a <strong>continuous trigger</strong> with the desired checkpoint interval as a parameter. For example,</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="n">spark</span> \ |
| <span class="p">.</span><span class="n">readStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"kafka"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="p">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"subscribe"</span><span class="p">,</span> <span class="s">"topic1"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">load</span><span class="p">()</span> \ |
| <span class="p">.</span><span class="n">selectExpr</span><span class="p">(</span><span class="s">"CAST(key AS STRING)"</span><span class="p">,</span> <span class="s">"CAST(value AS STRING)"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">writeStream</span> \ |
| <span class="p">.</span><span class="nb">format</span><span class="p">(</span><span class="s">"kafka"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="p">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"topic"</span><span class="p">,</span> <span class="s">"topic1"</span><span class="p">)</span> \ |
| <span class="p">.</span><span class="n">trigger</span><span class="p">(</span><span class="n">continuous</span><span class="o">=</span><span class="s">"1 second"</span><span class="p">)</span> \ <span class="c1"># only change in query |
| </span> <span class="p">.</span><span class="n">start</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.streaming.Trigger</span> |
| |
| <span class="n">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">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"subscribe"</span><span class="o">,</span> <span class="s">"topic1"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">load</span><span class="o">()</span> |
| <span class="o">.</span><span class="py">selectExpr</span><span class="o">(</span><span class="s">"CAST(key AS STRING)"</span><span class="o">,</span> <span class="s">"CAST(value AS STRING)"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">writeStream</span> |
| <span class="o">.</span><span class="py">format</span><span class="o">(</span><span class="s">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"topic"</span><span class="o">,</span> <span class="s">"topic1"</span><span class="o">)</span> |
| <span class="o">.</span><span class="py">trigger</span><span class="o">(</span><span class="nv">Trigger</span><span class="o">.</span><span class="py">Continuous</span><span class="o">(</span><span class="s">"1 second"</span><span class="o">))</span> <span class="c1">// only change in query</span> |
| <span class="o">.</span><span class="py">start</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="kn">import</span> <span class="nn">org.apache.spark.sql.streaming.Trigger</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="na">format</span><span class="o">(</span><span class="s">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"subscribe"</span><span class="o">,</span> <span class="s">"topic1"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">load</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">selectExpr</span><span class="o">(</span><span class="s">"CAST(key AS STRING)"</span><span class="o">,</span> <span class="s">"CAST(value AS STRING)"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">writeStream</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"kafka"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"kafka.bootstrap.servers"</span><span class="o">,</span> <span class="s">"host1:port1,host2:port2"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"topic"</span><span class="o">,</span> <span class="s">"topic1"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">trigger</span><span class="o">(</span><span class="nc">Trigger</span><span class="o">.</span><span class="na">Continuous</span><span class="o">(</span><span class="s">"1 second"</span><span class="o">))</span> <span class="c1">// only change in query</span> |
| <span class="o">.</span><span class="na">start</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>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.</p> |
| |
| <h2 class="no_toc" id="supported-queries">Supported Queries</h2> |
| |
| <p>As of Spark 2.4, only the following type of queries are supported in the continuous processing mode.</p> |
| |
| <ul> |
| <li><em>Operations</em>: Only map-like Dataset/DataFrame operations are supported in continuous mode, that is, only projections (<code class="language-plaintext highlighter-rouge">select</code>, <code class="language-plaintext highlighter-rouge">map</code>, <code class="language-plaintext highlighter-rouge">flatMap</code>, <code class="language-plaintext highlighter-rouge">mapPartitions</code>, etc.) and selections (<code class="language-plaintext highlighter-rouge">where</code>, <code class="language-plaintext highlighter-rouge">filter</code>, etc.). |
| <ul> |
| <li>All SQL functions are supported except aggregation functions (since aggregations are not yet supported), <code class="language-plaintext highlighter-rouge">current_timestamp()</code> and <code class="language-plaintext highlighter-rouge">current_date()</code> (deterministic computations using time is challenging).</li> |
| </ul> |
| </li> |
| <li><em>Sources</em>: |
| <ul> |
| <li>Kafka source: All options are supported.</li> |
| <li>Rate source: Good for testing. Only options that are supported in the continuous mode are <code class="language-plaintext highlighter-rouge">numPartitions</code> and <code class="language-plaintext highlighter-rouge">rowsPerSecond</code>.</li> |
| </ul> |
| </li> |
| <li><em>Sinks</em>: |
| <ul> |
| <li>Kafka sink: All options are supported.</li> |
| <li>Memory sink: Good for debugging.</li> |
| <li>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.</li> |
| </ul> |
| </li> |
| </ul> |
| |
| <p>See <a href="#input-sources">Input Sources</a> and <a href="#output-sinks">Output Sinks</a> 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.</p> |
| |
| <h2 class="no_toc" id="caveats-1">Caveats</h2> |
| |
| <ul> |
| <li>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.</li> |
| <li>Stopping a continuous processing stream may produce spurious task termination warnings. These can be safely ignored.</li> |
| <li>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.</li> |
| </ul> |
| |
| <h1 id="additional-information">Additional Information</h1> |
| |
| <p><strong>Notes</strong></p> |
| |
| <ul> |
| <li>Several configurations are not modifiable after the query has run. To change them, discard the checkpoint and start a new query. These configurations include: |
| <ul> |
| <li><code class="language-plaintext highlighter-rouge">spark.sql.shuffle.partitions</code> |
| <ul> |
| <li>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.</li> |
| <li>If you want to run fewer tasks for stateful operations, <code class="language-plaintext highlighter-rouge">coalesce</code> would help with avoiding unnecessary repartitioning. |
| <ul> |
| <li>After <code class="language-plaintext highlighter-rouge">coalesce</code>, the number of (reduced) tasks will be kept unless another shuffle happens.</li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><code class="language-plaintext highlighter-rouge">spark.sql.streaming.stateStore.providerClass</code>: To read the previous state of the query properly, the class of state store provider should be unchanged.</li> |
| <li><code class="language-plaintext highlighter-rouge">spark.sql.streaming.multipleWatermarkPolicy</code>: Modification of this would lead inconsistent watermark value when query contains multiple watermarks, hence the policy should be unchanged.</li> |
| </ul> |
| </li> |
| </ul> |
| |
| <p><strong>Further Reading</strong></p> |
| |
| <ul> |
| <li>See and run the |
| <a href="https://github.com/apache/spark/tree/v3.5.0/examples/src/main/scala/org/apache/spark/examples/sql/streaming">Scala</a>/<a href="https://github.com/apache/spark/tree/v3.5.0/examples/src/main/java/org/apache/spark/examples/sql/streaming">Java</a>/<a href="https://github.com/apache/spark/tree/v3.5.0/examples/src/main/python/sql/streaming">Python</a>/<a href="https://github.com/apache/spark/tree/v3.5.0/examples/src/main/r/streaming">R</a> |
| examples. |
| <ul> |
| <li><a href="index.html#running-the-examples-and-shell">Instructions</a> on how to run Spark examples</li> |
| </ul> |
| </li> |
| <li>Read about integrating with Kafka in the <a href="structured-streaming-kafka-integration.html">Structured Streaming Kafka Integration Guide</a></li> |
| <li>Read more details about using DataFrames/Datasets in the <a href="sql-programming-guide.html">Spark SQL Programming Guide</a></li> |
| <li>Third-party Blog Posts |
| <ul> |
| <li><a href="https://databricks.com/blog/2017/01/19/real-time-streaming-etl-structured-streaming-apache-spark-2-1.html">Real-time Streaming ETL with Structured Streaming in Apache Spark 2.1 (Databricks Blog)</a></li> |
| <li><a href="https://databricks.com/blog/2017/04/04/real-time-end-to-end-integration-with-apache-kafka-in-apache-sparks-structured-streaming.html">Real-Time End-to-End Integration with Apache Kafka in Apache Spark’s Structured Streaming (Databricks Blog)</a></li> |
| <li><a href="https://databricks.com/blog/2017/05/08/event-time-aggregation-watermarking-apache-sparks-structured-streaming.html">Event-time Aggregation and Watermarking in Apache Spark’s Structured Streaming (Databricks Blog)</a></li> |
| </ul> |
| </li> |
| </ul> |
| |
| <p><strong>Talks</strong></p> |
| |
| <ul> |
| <li>Spark Summit Europe 2017 |
| <ul> |
| <li>Easy, Scalable, Fault-tolerant Stream Processing with Structured Streaming in Apache Spark - |
| <a href="https://databricks.com/session/easy-scalable-fault-tolerant-stream-processing-with-structured-streaming-in-apache-spark">Part 1 slides/video</a>, <a href="https://databricks.com/session/easy-scalable-fault-tolerant-stream-processing-with-structured-streaming-in-apache-spark-continues">Part 2 slides/video</a></li> |
| <li>Deep Dive into Stateful Stream Processing in Structured Streaming - <a href="https://databricks.com/session/deep-dive-into-stateful-stream-processing-in-structured-streaming">slides/video</a></li> |
| </ul> |
| </li> |
| <li>Spark Summit 2016 |
| <ul> |
| <li>A Deep Dive into Structured Streaming - <a href="https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/">slides/video</a></li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h1 id="migration-guide">Migration Guide</h1> |
| |
| <p>The migration guide is now archived <a href="ss-migration-guide.html">on this page</a>.</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:3.5.0"] |
| }, |
| 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> |