blob: c9793d1f559e1e18e15a51fe6ea4db0fd0a9e190 [file] [log] [blame]
<!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">
<title>Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) - Spark 3.4.3 Documentation</title>
<link rel="stylesheet" href="css/bootstrap.min.css">
<style>
body {
padding-top: 60px;
padding-bottom: 40px;
}
</style>
<meta name="viewport" content="width=device-width">
<link rel="stylesheet" href="css/main.css">
<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="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/docsearch.min.css" />
<link rel="stylesheet" href="css/docsearch.css">
<!-- Matomo -->
<script>
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
_paq.push(["disableCookies"]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="https://analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '40']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
<!-- End Matomo Code -->
</head>
<body>
<!--[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 fixed-top navbar-expand-md navbar-light bg-light" id="topbar">
<div class="container">
<div class="navbar-header">
<div class="navbar-brand"><a href="index.html">
<img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">3.4.3</span>
</div>
</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">
<!--TODO(andyk): Add class="active" attribute to li some how.-->
<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.4.3</span></span>-->
</div>
</div>
</nav>
<div class="container-wrapper">
<div class="content mr-3" id="content">
<h1 class="title">Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher)</h1>
<p>Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka.</p>
<h2 id="linking">Linking</h2>
<p>For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>groupId = org.apache.spark
artifactId = spark-sql-kafka-0-10_2.12
version = 3.4.3
</code></pre></div></div>
<p>Please note that to use the headers functionality, your Kafka client version should be version 0.11.0.0 or up.</p>
<p>For Python applications, you need to add this above library and its dependencies when deploying your
application. See the <a href="#deploying">Deploying</a> subsection below.</p>
<p>For experimenting on <code class="language-plaintext highlighter-rouge">spark-shell</code>, you need to add this above library and its dependencies too when invoking <code class="language-plaintext highlighter-rouge">spark-shell</code>. Also, see the <a href="#deploying">Deploying</a> subsection below.</p>
<h2 id="reading-data-from-kafka">Reading Data from Kafka</h2>
<h3 id="creating-a-kafka-source-for-streaming-queries">Creating a Kafka Source for Streaming Queries</h3>
<div class="codetabs">
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Subscribe to 1 topic</span>
<span class="k">val</span> <span class="nv">df</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">"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="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">)]</span>
<span class="c1">// Subscribe to 1 topic, with headers</span>
<span class="k">val</span> <span class="nv">df</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">"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">option</span><span class="o">(</span><span class="s">"includeHeaders"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="nv">df</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="s">"headers"</span><span class="o">)</span>
<span class="o">.</span><span class="py">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</span>, <span class="kt">Array</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">])])]</span>
<span class="c1">// Subscribe to multiple topics</span>
<span class="k">val</span> <span class="nv">df</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">"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,topic2"</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">)]</span>
<span class="c1">// Subscribe to a pattern</span>
<span class="k">val</span> <span class="nv">df</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">"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">"subscribePattern"</span><span class="o">,</span> <span class="s">"topic.*"</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</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">// Subscribe to 1 topic</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</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">"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="n">df</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="c1">// Subscribe to 1 topic, with headers</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</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">"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">option</span><span class="o">(</span><span class="s">"includeHeaders"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">()</span>
<span class="n">df</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="s">"headers"</span><span class="o">);</span>
<span class="c1">// Subscribe to multiple topics</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</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">"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,topic2"</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">();</span>
<span class="n">df</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="c1">// Subscribe to a pattern</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</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">"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">"subscribePattern"</span><span class="o">,</span> <span class="s">"topic.*"</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">();</span>
<span class="n">df</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></code></pre></figure>
</div>
<div data-lang="python">
<figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="c1"># Subscribe to 1 topic
</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">"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="n">df</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="c1"># Subscribe to 1 topic, with headers
</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">"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">option</span><span class="p">(</span><span class="s">"includeHeaders"</span><span class="p">,</span> <span class="s">"true"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">load</span><span class="p">()</span>
<span class="n">df</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="s">"headers"</span><span class="p">)</span>
<span class="c1"># Subscribe to multiple topics
</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">"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,topic2"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">load</span><span class="p">()</span>
<span class="n">df</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="c1"># Subscribe to a pattern
</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">"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">"subscribePattern"</span><span class="p">,</span> <span class="s">"topic.*"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">load</span><span class="p">()</span>
<span class="n">df</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></code></pre></figure>
</div>
</div>
<h3 id="creating-a-kafka-source-for-batch-queries">Creating a Kafka Source for Batch Queries</h3>
<p>If you have a use case that is better suited to batch processing,
you can create a Dataset/DataFrame for a defined range of offsets.</p>
<div class="codetabs">
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Subscribe to 1 topic defaults to the earliest and latest offsets</span>
<span class="k">val</span> <span class="nv">df</span> <span class="k">=</span> <span class="n">spark</span>
<span class="o">.</span><span class="py">read</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="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">)]</span>
<span class="c1">// Subscribe to multiple topics, specifying explicit Kafka offsets</span>
<span class="k">val</span> <span class="nv">df</span> <span class="k">=</span> <span class="n">spark</span>
<span class="o">.</span><span class="py">read</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,topic2"</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"startingOffsets"</span><span class="o">,</span> <span class="s">"""{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}"""</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"endingOffsets"</span><span class="o">,</span> <span class="s">"""{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}"""</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">)]</span>
<span class="c1">// Subscribe to a pattern, at the earliest and latest offsets</span>
<span class="k">val</span> <span class="nv">df</span> <span class="k">=</span> <span class="n">spark</span>
<span class="o">.</span><span class="py">read</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">"subscribePattern"</span><span class="o">,</span> <span class="s">"topic.*"</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"startingOffsets"</span><span class="o">,</span> <span class="s">"earliest"</span><span class="o">)</span>
<span class="o">.</span><span class="py">option</span><span class="o">(</span><span class="s">"endingOffsets"</span><span class="o">,</span> <span class="s">"latest"</span><span class="o">)</span>
<span class="o">.</span><span class="py">load</span><span class="o">()</span>
<span class="nv">df</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">as</span><span class="o">[(</span><span class="kt">String</span>, <span class="kt">String</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">// Subscribe to 1 topic defaults to the earliest and latest offsets</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</span> <span class="n">df</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="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="n">df</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="c1">// Subscribe to multiple topics, specifying explicit Kafka offsets</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</span> <span class="n">df</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="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,topic2"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"startingOffsets"</span><span class="o">,</span> <span class="s">"{\"topic1\":{\"0\":23,\"1\":-2},\"topic2\":{\"0\":-2}}"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"endingOffsets"</span><span class="o">,</span> <span class="s">"{\"topic1\":{\"0\":50,\"1\":-1},\"topic2\":{\"0\":-1}}"</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">();</span>
<span class="n">df</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="c1">// Subscribe to a pattern, at the earliest and latest offsets</span>
<span class="nc">Dataset</span><span class="o">&lt;</span><span class="nc">Row</span><span class="o">&gt;</span> <span class="n">df</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="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">"subscribePattern"</span><span class="o">,</span> <span class="s">"topic.*"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"startingOffsets"</span><span class="o">,</span> <span class="s">"earliest"</span><span class="o">)</span>
<span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"endingOffsets"</span><span class="o">,</span> <span class="s">"latest"</span><span class="o">)</span>
<span class="o">.</span><span class="na">load</span><span class="o">();</span>
<span class="n">df</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></code></pre></figure>
</div>
<div data-lang="python">
<figure class="highlight"><pre><code class="language-python" data-lang="python"><span class="c1"># Subscribe to 1 topic defaults to the earliest and latest offsets
</span><span class="n">df</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="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="n">df</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="c1"># Subscribe to multiple topics, specifying explicit Kafka offsets
</span><span class="n">df</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="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,topic2"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"startingOffsets"</span><span class="p">,</span> <span class="s">"""{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}"""</span><span class="p">)</span> \
<span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"endingOffsets"</span><span class="p">,</span> <span class="s">"""{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}"""</span><span class="p">)</span> \
<span class="p">.</span><span class="n">load</span><span class="p">()</span>
<span class="n">df</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="c1"># Subscribe to a pattern, at the earliest and latest offsets
</span><span class="n">df</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="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">"subscribePattern"</span><span class="p">,</span> <span class="s">"topic.*"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"startingOffsets"</span><span class="p">,</span> <span class="s">"earliest"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">option</span><span class="p">(</span><span class="s">"endingOffsets"</span><span class="p">,</span> <span class="s">"latest"</span><span class="p">)</span> \
<span class="p">.</span><span class="n">load</span><span class="p">()</span>
<span class="n">df</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></code></pre></figure>
</div>
</div>
<p>Each row in the source has the following schema:</p>
<table class="table">
<tr><th>Column</th><th>Type</th></tr>
<tr>
<td>key</td>
<td>binary</td>
</tr>
<tr>
<td>value</td>
<td>binary</td>
</tr>
<tr>
<td>topic</td>
<td>string</td>
</tr>
<tr>
<td>partition</td>
<td>int</td>
</tr>
<tr>
<td>offset</td>
<td>long</td>
</tr>
<tr>
<td>timestamp</td>
<td>timestamp</td>
</tr>
<tr>
<td>timestampType</td>
<td>int</td>
</tr>
<tr>
<td>headers (optional)</td>
<td>array</td>
</tr>
</table>
<p>The following options must be set for the Kafka source
for both batch and streaming queries.</p>
<table class="table">
<tr><th>Option</th><th>value</th><th>meaning</th></tr>
<tr>
<td>assign</td>
<td>json string {"topicA":[0,1],"topicB":[2,4]}</td>
<td>Specific TopicPartitions to consume.
Only one of "assign", "subscribe" or "subscribePattern"
options can be specified for Kafka source.</td>
</tr>
<tr>
<td>subscribe</td>
<td>A comma-separated list of topics</td>
<td>The topic list to subscribe.
Only one of "assign", "subscribe" or "subscribePattern"
options can be specified for Kafka source.</td>
</tr>
<tr>
<td>subscribePattern</td>
<td>Java regex string</td>
<td>The pattern used to subscribe to topic(s).
Only one of "assign, "subscribe" or "subscribePattern"
options can be specified for Kafka source.</td>
</tr>
<tr>
<td>kafka.bootstrap.servers</td>
<td>A comma-separated list of host:port</td>
<td>The Kafka "bootstrap.servers" configuration.</td>
</tr>
</table>
<p>The following configurations are optional:</p>
<table class="table">
<tr><th>Option</th><th>value</th><th>default</th><th>query type</th><th>meaning</th></tr>
<tr>
<td>startingTimestamp</td>
<td>timestamp string e.g. "1000"</td>
<td>none (next preference is <code>startingOffsetsByTimestamp</code>)</td>
<td>streaming and batch</td>
<td>The start point of timestamp when a query is started, a string specifying a starting timestamp for
all partitions in topics being subscribed. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the behavior will follow to the value of the option <code>startingOffsetsByTimestampStrategy</code><p />
<p />
Note1: <code>startingTimestamp</code> takes precedence over <code>startingOffsetsByTimestamp</code> and <code>startingOffsets</code>.<p />
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
</tr>
<tr>
<td>startingOffsetsByTimestamp</td>
<td>json string
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
</td>
<td>none (next preference is <code>startingOffsets</code>)</td>
<td>streaming and batch</td>
<td>The start point of timestamp when a query is started, a json string specifying a starting timestamp for
each TopicPartition. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the behavior will follow to the value of the option <code>startingOffsetsByTimestampStrategy</code><p />
<p />
Note1: <code>startingOffsetsByTimestamp</code> takes precedence over <code>startingOffsets</code>.<p />
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
</tr>
<tr>
<td>startingOffsets</td>
<td>"earliest", "latest" (streaming only), or json string
""" {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} """
</td>
<td>"latest" for streaming, "earliest" for batch</td>
<td>streaming and batch</td>
<td>The start point when a query is started, either "earliest" which is from the earliest offsets,
"latest" which is just from the latest offsets, or a json string specifying a starting offset for
each TopicPartition. In the json, -2 as an offset can be used to refer to earliest, -1 to latest.
Note: For batch queries, latest (either implicitly or by using -1 in json) is not allowed.
For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
</tr>
<tr>
<td>endingTimestamp</td>
<td>timestamp string e.g. "1000"</td>
<td>none (next preference is <code>endingOffsetsByTimestamp</code>)</td>
<td>batch query</td>
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for
all partitions in topics being subscribed. Please refer the details on timestamp offset options below.
If Kafka doesn't return the matched offset, the offset will be set to latest.<p />
Note: <code>endingTimestamp</code> takes precedence over <code>endingOffsetsByTimestamp</code> and <code>endingOffsets</code>.<p />
</td>
</tr>
<tr>
<td>endingOffsetsByTimestamp</td>
<td>json string
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
</td>
<td>none (next preference is <code>endingOffsets</code>)</td>
<td>batch query</td>
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for each TopicPartition.
Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the offset will be set to latest.<p />
Note: <code>endingOffsetsByTimestamp</code> takes precedence over <code>endingOffsets</code>.
</td>
</tr>
<tr>
<td>endingOffsets</td>
<td>latest or json string
{"topicA":{"0":23,"1":-1},"topicB":{"0":-1}}
</td>
<td>latest</td>
<td>batch query</td>
<td>The end point when a batch query is ended, either "latest" which is just referred to the
latest, or a json string specifying an ending offset for each TopicPartition. In the json, -1
as an offset can be used to refer to latest, and -2 (earliest) as an offset is not allowed.</td>
</tr>
<tr>
<td>failOnDataLoss</td>
<td>true or false</td>
<td>true</td>
<td>streaming and batch</td>
<td>Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or
offsets are out of range). This may be a false alarm. You can disable it when it doesn't work
as you expected.</td>
</tr>
<tr>
<td>kafkaConsumer.pollTimeoutMs</td>
<td>long</td>
<td>120000</td>
<td>streaming and batch</td>
<td>The timeout in milliseconds to poll data from Kafka in executors. When not defined it falls
back to <code>spark.network.timeout</code>.</td>
</tr>
<tr>
<td>fetchOffset.numRetries</td>
<td>int</td>
<td>3</td>
<td>streaming and batch</td>
<td>Number of times to retry before giving up fetching Kafka offsets.</td>
</tr>
<tr>
<td>fetchOffset.retryIntervalMs</td>
<td>long</td>
<td>10</td>
<td>streaming and batch</td>
<td>milliseconds to wait before retrying to fetch Kafka offsets</td>
</tr>
<tr>
<td>maxOffsetsPerTrigger</td>
<td>long</td>
<td>none</td>
<td>streaming query</td>
<td>Rate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume.</td>
</tr>
<tr>
<td>minOffsetsPerTrigger</td>
<td>long</td>
<td>none</td>
<td>streaming query</td>
<td>Minimum number of offsets to be processed per trigger interval. The specified total number of
offsets will be proportionally split across topicPartitions of different volume. Note, if the
maxTriggerDelay is exceeded, a trigger will be fired even if the number of available offsets
doesn't reach minOffsetsPerTrigger.</td>
</tr>
<tr>
<td>maxTriggerDelay</td>
<td>time with units</td>
<td>15m</td>
<td>streaming query</td>
<td>Maximum amount of time for which trigger can be delayed between two triggers provided some
data is available from the source. This option is only applicable if minOffsetsPerTrigger is set.</td>
</tr>
<tr>
<td>minPartitions</td>
<td>int</td>
<td>none</td>
<td>streaming and batch</td>
<td>Desired minimum number of partitions to read from Kafka.
By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka.
If you set this option to a value greater than your topicPartitions, Spark will divvy up large
Kafka partitions to smaller pieces. Please note that this configuration is like a <code>hint</code>: the
number of Spark tasks will be <strong>approximately</strong> <code>minPartitions</code>. It can be less or more depending on
rounding errors or Kafka partitions that didn't receive any new data.</td>
</tr>
<tr>
<td>groupIdPrefix</td>
<td>string</td>
<td>spark-kafka-source</td>
<td>streaming and batch</td>
<td>Prefix of consumer group identifiers (<code>group.id</code>) that are generated by structured streaming
queries. If "kafka.group.id" is set, this option will be ignored.</td>
</tr>
<tr>
<td>kafka.group.id</td>
<td>string</td>
<td>none</td>
<td>streaming and batch</td>
<td>The Kafka group id to use in Kafka consumer while reading from Kafka. Use this with caution.
By default, each query generates a unique group id for reading data. This ensures that each Kafka
source has its own consumer group that does not face interference from any other consumer, and
therefore can read all of the partitions of its subscribed topics. In some scenarios (for example,
Kafka group-based authorization), you may want to use a specific authorized group id to read data.
You can optionally set the group id. However, do this with extreme caution as it can cause
unexpected behavior. Concurrently running queries (both, batch and streaming) or sources with the
same group id are likely interfere with each other causing each query to read only part of the
data. This may also occur when queries are started/restarted in quick succession. To minimize such
issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to
be very small. When this is set, option "groupIdPrefix" will be ignored.</td>
</tr>
<tr>
<td>includeHeaders</td>
<td>boolean</td>
<td>false</td>
<td>streaming and batch</td>
<td>Whether to include the Kafka headers in the row.</td>
</tr>
<tr>
<td>startingOffsetsByTimestampStrategy</td>
<td>"error" or "latest"</td>
<td>"error"</td>
<td>streaming and batch</td>
<td>The strategy will be used when the specified starting offset by timestamp (either global or per partition) doesn't match with the offset Kafka returned. Here's the strategy name and corresponding descriptions:<p />
<p />
"error": fail the query and end users have to deal with workarounds requiring manual steps.<p />
"latest": assigns the latest offset for these partitions, so that Spark can read newer records from these partitions in further micro-batches.<p /></td>
</tr>
</table>
<h3 id="details-on-timestamp-offset-options">Details on timestamp offset options</h3>
<p>The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition.
The behavior varies across options if Kafka doesn&#8217;t return the matched offset - check the description of each option.</p>
<p>Spark simply passes the timestamp information to <code>KafkaConsumer.offsetsForTimes</code>, and doesn&#8217;t interpret or reason about the value.
For more details on <code>KafkaConsumer.offsetsForTimes</code>, please refer <a href="http://kafka.apache.org/0101/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes(java.util.Map)">javadoc</a> for details.
Also, the meaning of <code>timestamp</code> here can be vary according to Kafka configuration (<code>log.message.timestamp.type</code>): please refer <a href="https://kafka.apache.org/documentation/">Kafka documentation</a> for further details.</p>
<p>Timestamp offset options require Kafka 0.10.1.0 or higher.</p>
<h3 id="offset-fetching">Offset fetching</h3>
<p>In Spark 3.0 and before Spark uses <code>KafkaConsumer</code> for offset fetching which could cause infinite wait in the driver.
In Spark 3.1 a new configuration option added <code>spark.sql.streaming.kafka.useDeprecatedOffsetFetching</code> (default: <code>false</code>)
which allows Spark to use new offset fetching mechanism using <code>AdminClient</code>. (Set this to <code class="language-plaintext highlighter-rouge">true</code> to use old offset fetching with <code>KafkaConsumer</code>.)</p>
<p>When the new mechanism used the following applies.</p>
<p>First of all the new approach supports Kafka brokers <code class="language-plaintext highlighter-rouge">0.11.0.0+</code>.</p>
<p>In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:</p>
<ul>
<li>Topic resource describe operation</li>
<li>Topic resource read operation</li>
<li>Group resource read operation</li>
</ul>
<p>Since Spark 3.1, offsets can be obtained with <code>AdminClient</code> instead of <code>KafkaConsumer</code> and for that the following ACLs needed from driver perspective:</p>
<ul>
<li>Topic resource describe operation</li>
</ul>
<p>Since <code>AdminClient</code> in driver is not connecting to consumer group, <code>group.id</code> based authorization will not work anymore (executors never done group based authorization).
Worth to mention executor side is behaving the exact same way like before (group prefix and override works).</p>
<h3 id="consumer-caching">Consumer Caching</h3>
<p>It&#8217;s time-consuming to initialize Kafka consumers, especially in streaming scenarios where processing time is a key factor.
Because of this, Spark pools Kafka consumers on executors, by leveraging Apache Commons Pool.</p>
<p>The caching key is built up from the following information:</p>
<ul>
<li>Topic name</li>
<li>Topic partition</li>
<li>Group ID</li>
</ul>
<p>The following properties are available to configure the consumer pool:</p>
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td>spark.kafka.consumer.cache.capacity</td>
<td>64</td>
<td>The maximum number of consumers cached. Please note that it's a soft limit.</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.timeout</td>
<td>5m (5 minutes)</td>
<td>The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor.</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run.</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.jmx.enable</td>
<td>false</td>
<td>Enable or disable JMX for pools created with this configuration instance. Statistics of the pool are available via JMX instance.
The prefix of JMX name is set to "kafka010-cached-simple-kafka-consumer-pool".
</td>
<td>3.0.0</td>
</tr>
</table>
<p>The size of the pool is limited by <code>spark.kafka.consumer.cache.capacity</code>,
but it works as &#8220;soft-limit&#8221; to not block Spark tasks.</p>
<p>Idle eviction thread periodically removes consumers which are not used longer than given timeout.
If this threshold is reached when borrowing, it tries to remove the least-used entry that is currently not in use.</p>
<p>If it cannot be removed, then the pool will keep growing. In the worst case, the pool will grow to
the max number of concurrent tasks that can run in the executor (that is, number of task slots).</p>
<p>If a task fails for any reason, the new task is executed with a newly created Kafka consumer for safety reasons.
At the same time, we invalidate all consumers in pool which have same caching key, to remove consumer which was used
in failed execution. Consumers which any other tasks are using will not be closed, but will be invalidated as well
when they are returned into pool.</p>
<p>Along with consumers, Spark pools the records fetched from Kafka separately, to let Kafka consumers stateless in point
of Spark&#8217;s view, and maximize the efficiency of pooling. It leverages same cache key with Kafka consumers pool.
Note that it doesn&#8217;t leverage Apache Commons Pool due to the difference of characteristics.</p>
<p>The following properties are available to configure the fetched data pool:</p>
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td>spark.kafka.consumer.fetchedData.cache.timeout</td>
<td>5m (5 minutes)</td>
<td>The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor.</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run.</td>
<td>3.0.0</td>
</tr>
</table>
<h2 id="writing-data-to-kafka">Writing Data to Kafka</h2>
<p>Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that
Apache Kafka only supports at least once write semantics. Consequently, when writing&#8212;either Streaming Queries
or Batch Queries&#8212;to Kafka, some records may be duplicated; this can happen, for example, if Kafka needs
to retry a message that was not acknowledged by a Broker, even though that Broker received and wrote the message record.
Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However,
if writing the query is successful, then you can assume that the query output was written at least once. A possible
solution to remove duplicates when reading the written data could be to introduce a primary (unique) key
that can be used to perform de-duplication when reading.</p>
<p>The Dataframe being written to Kafka should have the following columns in schema:</p>
<table class="table">
<tr><th>Column</th><th>Type</th></tr>
<tr>
<td>key (optional)</td>
<td>string or binary</td>
</tr>
<tr>
<td>value (required)</td>
<td>string or binary</td>
</tr>
<tr>
<td>headers (optional)</td>
<td>array</td>
</tr>
<tr>
<td>topic (*optional)</td>
<td>string</td>
</tr>
<tr>
<td>partition (optional)</td>
<td>int</td>
</tr>
</table>
<p>* The topic column is required if the &#8220;topic&#8221; configuration option is not specified.<br /></p>
<p>The value column is the only required option. If a key column is not specified then
a <code class="language-plaintext highlighter-rouge">null</code> valued key column will be automatically added (see Kafka semantics on
how <code class="language-plaintext highlighter-rouge">null</code> valued key values are handled). If a topic column exists then its value
is used as the topic when writing the given row to Kafka, unless the &#8220;topic&#8221; configuration
option is set i.e., the &#8220;topic&#8221; configuration option overrides the topic column.
If a &#8220;partition&#8221; column is not specified (or its value is <code class="language-plaintext highlighter-rouge">null</code>)
then the partition is calculated by the Kafka producer.
A Kafka partitioner can be specified in Spark by setting the
<code class="language-plaintext highlighter-rouge">kafka.partitioner.class</code> option. If not present, Kafka default partitioner
will be used.</p>
<p>The following options must be set for the Kafka sink
for both batch and streaming queries.</p>
<table class="table">
<tr><th>Option</th><th>value</th><th>meaning</th></tr>
<tr>
<td>kafka.bootstrap.servers</td>
<td>A comma-separated list of host:port</td>
<td>The Kafka "bootstrap.servers" configuration.</td>
</tr>
</table>
<p>The following configurations are optional:</p>
<table class="table">
<tr><th>Option</th><th>value</th><th>default</th><th>query type</th><th>meaning</th></tr>
<tr>
<td>topic</td>
<td>string</td>
<td>none</td>
<td>streaming and batch</td>
<td>Sets the topic that all rows will be written to in Kafka. This option overrides any
topic column that may exist in the data.</td>
</tr>
<tr>
<td>includeHeaders</td>
<td>boolean</td>
<td>false</td>
<td>streaming and batch</td>
<td>Whether to include the Kafka headers in the row.</td>
</tr>
</table>
<h3 id="creating-a-kafka-sink-for-streaming-queries">Creating a Kafka Sink for Streaming Queries</h3>
<div class="codetabs">
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Write key-value data from a DataFrame to a specific Kafka topic specified in an option</span>
<span class="k">val</span> <span class="nv">ds</span> <span class="k">=</span> <span class="n">df</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">start</span><span class="o">()</span>
<span class="c1">// Write key-value data from a DataFrame to Kafka using a topic specified in the data</span>
<span class="k">val</span> <span class="nv">ds</span> <span class="k">=</span> <span class="n">df</span>
<span class="o">.</span><span class="py">selectExpr</span><span class="o">(</span><span class="s">"topic"</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">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="c1">// Write key-value data from a DataFrame to a specific Kafka topic specified in an option</span>
<span class="nc">StreamingQuery</span> <span class="n">ds</span> <span class="o">=</span> <span class="n">df</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="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">start</span><span class="o">();</span>
<span class="c1">// Write key-value data from a DataFrame to Kafka using a topic specified in the data</span>
<span class="nc">StreamingQuery</span> <span class="n">ds</span> <span class="o">=</span> <span class="n">df</span>
<span class="o">.</span><span class="na">selectExpr</span><span class="o">(</span><span class="s">"topic"</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="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">start</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="c1"># Write key-value data from a DataFrame to a specific Kafka topic specified in an option
</span><span class="n">ds</span> <span class="o">=</span> <span class="n">df</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">start</span><span class="p">()</span>
<span class="c1"># Write key-value data from a DataFrame to Kafka using a topic specified in the data
</span><span class="n">ds</span> <span class="o">=</span> <span class="n">df</span> \
<span class="p">.</span><span class="n">selectExpr</span><span class="p">(</span><span class="s">"topic"</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">start</span><span class="p">()</span></code></pre></figure>
</div>
</div>
<h3 id="writing-the-output-of-batch-queries-to-kafka">Writing the output of Batch Queries to Kafka</h3>
<div class="codetabs">
<div data-lang="scala">
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Write key-value data from a DataFrame to a specific Kafka topic specified in an option</span>
<span class="nv">df</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">write</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">save</span><span class="o">()</span>
<span class="c1">// Write key-value data from a DataFrame to Kafka using a topic specified in the data</span>
<span class="nv">df</span><span class="o">.</span><span class="py">selectExpr</span><span class="o">(</span><span class="s">"topic"</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">write</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">save</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">// Write key-value data from a DataFrame to a specific Kafka topic specified in an option</span>
<span class="n">df</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">write</span><span class="o">()</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">save</span><span class="o">();</span>
<span class="c1">// Write key-value data from a DataFrame to Kafka using a topic specified in the data</span>
<span class="n">df</span><span class="o">.</span><span class="na">selectExpr</span><span class="o">(</span><span class="s">"topic"</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">write</span><span class="o">()</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">save</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="c1"># Write key-value data from a DataFrame to a specific Kafka topic specified in an option
</span><span class="n">df</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">write</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">save</span><span class="p">()</span>
<span class="c1"># Write key-value data from a DataFrame to Kafka using a topic specified in the data
</span><span class="n">df</span><span class="p">.</span><span class="n">selectExpr</span><span class="p">(</span><span class="s">"topic"</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">write</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">save</span><span class="p">()</span></code></pre></figure>
</div>
</div>
<h3 id="producer-caching">Producer Caching</h3>
<p>Given Kafka producer instance is designed to be thread-safe, Spark initializes a Kafka producer instance and co-use across tasks for same caching key.</p>
<p>The caching key is built up from the following information:</p>
<ul>
<li>Kafka producer configuration</li>
</ul>
<p>This includes configuration for authorization, which Spark will automatically include when delegation token is being used. Even we take authorization into account, you can expect same Kafka producer instance will be used among same Kafka producer configuration.
It will use different Kafka producer when delegation token is renewed; Kafka producer instance for old delegation token will be evicted according to the cache policy.</p>
<p>The following properties are available to configure the producer pool:</p>
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td>spark.kafka.producer.cache.timeout</td>
<td>10m (10 minutes)</td>
<td>The minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor.</td>
<td>2.2.1</td>
</tr>
<tr>
<td>spark.kafka.producer.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run.</td>
<td>3.0.0</td>
</tr>
</table>
<p>Idle eviction thread periodically removes producers which are not used longer than given timeout. Note that the producer is shared and used concurrently, so the last used timestamp is determined by the moment the producer instance is returned and reference count is 0.</p>
<h2 id="kafka-specific-configurations">Kafka Specific Configurations</h2>
<p>Kafka&#8217;s own configurations can be set via <code class="language-plaintext highlighter-rouge">DataStreamReader.option</code> with <code class="language-plaintext highlighter-rouge">kafka.</code> prefix, e.g,
<code class="language-plaintext highlighter-rouge">stream.option("kafka.bootstrap.servers", "host:port")</code>. For possible kafka parameters, see
<a href="http://kafka.apache.org/documentation.html#consumerconfigs">Kafka consumer config docs</a> for
parameters related to reading data, and <a href="http://kafka.apache.org/documentation/#producerconfigs">Kafka producer config docs</a>
for parameters related to writing data.</p>
<p>Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception:</p>
<ul>
<li><strong>group.id</strong>: Kafka source will create a unique group id for each query automatically. The user can
set the prefix of the automatically generated group.id&#8217;s via the optional source option <code class="language-plaintext highlighter-rouge">groupIdPrefix</code>,
default value is &#8220;spark-kafka-source&#8221;. You can also set &#8220;kafka.group.id&#8221; to force Spark to use a special
group id, however, please read warnings for this option and use it with caution.</li>
<li><strong>auto.offset.reset</strong>: Set the source option <code class="language-plaintext highlighter-rouge">startingOffsets</code> to specify
where to start instead. Structured Streaming manages which offsets are consumed internally, rather
than rely on the kafka Consumer to do it. This will ensure that no data is missed when new
topics/partitions are dynamically subscribed. Note that <code class="language-plaintext highlighter-rouge">startingOffsets</code> only applies when a new
streaming query is started, and that resuming will always pick up from where the query left off. Note
that when the offsets consumed by a streaming application no longer exist in Kafka (e.g., topics are deleted,
offsets are out of range, or offsets are removed after retention period), the offsets will not be reset
and the streaming application will see data loss. In extreme cases, for example the throughput of the
streaming application cannot catch up the retention speed of Kafka, the input rows of a batch might be
gradually reduced until zero when the offset ranges of the batch are completely not in Kafka. Enabling
<code class="language-plaintext highlighter-rouge">failOnDataLoss</code> option can ask Structured Streaming to fail the query for such cases.</li>
<li><strong>key.deserializer</strong>: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use
DataFrame operations to explicitly deserialize the keys.</li>
<li><strong>value.deserializer</strong>: Values are always deserialized as byte arrays with ByteArrayDeserializer.
Use DataFrame operations to explicitly deserialize the values.</li>
<li><strong>key.serializer</strong>: Keys are always serialized with ByteArraySerializer or StringSerializer. Use
DataFrame operations to explicitly serialize the keys into either strings or byte arrays.</li>
<li><strong>value.serializer</strong>: values are always serialized with ByteArraySerializer or StringSerializer. Use
DataFrame operations to explicitly serialize the values into either strings or byte arrays.</li>
<li><strong>enable.auto.commit</strong>: Kafka source doesn&#8217;t commit any offset.</li>
<li><strong>interceptor.classes</strong>: Kafka source always read keys and values as byte arrays. It&#8217;s not safe to
use ConsumerInterceptor as it may break the query.</li>
</ul>
<h2 id="deploying">Deploying</h2>
<p>As with any Spark applications, <code class="language-plaintext highlighter-rouge">spark-submit</code> is used to launch your application. <code class="language-plaintext highlighter-rouge">spark-sql-kafka-0-10_2.12</code>
and its dependencies can be directly added to <code class="language-plaintext highlighter-rouge">spark-submit</code> using <code class="language-plaintext highlighter-rouge">--packages</code>, such as,</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_2.12:3.4.3 ...
</code></pre></div></div>
<p>For experimenting on <code class="language-plaintext highlighter-rouge">spark-shell</code>, you can also use <code class="language-plaintext highlighter-rouge">--packages</code> to add <code class="language-plaintext highlighter-rouge">spark-sql-kafka-0-10_2.12</code> and its dependencies directly,</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_2.12:3.4.3 ...
</code></pre></div></div>
<p>See <a href="submitting-applications.html">Application Submission Guide</a> for more details about submitting
applications with external dependencies.</p>
<h2 id="security">Security</h2>
<p>Kafka 0.9.0.0 introduced several features that increases security in a cluster. For detailed
description about these possibilities, see <a href="http://kafka.apache.org/documentation.html#security">Kafka security docs</a>.</p>
<p>It&#8217;s worth noting that security is optional and turned off by default.</p>
<p>Spark supports the following ways to authenticate against Kafka cluster:</p>
<ul>
<li><strong>Delegation token (introduced in Kafka broker 1.1.0)</strong></li>
<li><strong>JAAS login configuration</strong></li>
</ul>
<h3 id="delegation-token">Delegation token</h3>
<p>This way the application can be configured via Spark parameters and may not need JAAS login
configuration (Spark can use Kafka&#8217;s dynamic JAAS configuration feature). For further information
about delegation tokens, see <a href="http://kafka.apache.org/documentation/#security_delegation_token">Kafka delegation token docs</a>.</p>
<p>The process is initiated by Spark&#8217;s Kafka delegation token provider. When <code class="language-plaintext highlighter-rouge">spark.kafka.clusters.${cluster}.auth.bootstrap.servers</code> is set,
Spark considers the following log in options, in order of preference:</p>
<ul>
<li><strong>JAAS login configuration</strong>, please see example below.</li>
<li>
<p><strong>Keytab file</strong>, such as,</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-submit \
--keytab &lt;KEYTAB_FILE&gt; \
--principal &lt;PRINCIPAL&gt; \
--conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=&lt;KAFKA_SERVERS&gt; \
...
</code></pre></div> </div>
</li>
<li>
<p><strong>Kerberos credential cache</strong>, such as,</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-submit \
--conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=&lt;KAFKA_SERVERS&gt; \
...
</code></pre></div> </div>
</li>
</ul>
<p>The Kafka delegation token provider can be turned off by setting <code class="language-plaintext highlighter-rouge">spark.security.credentials.kafka.enabled</code> to <code class="language-plaintext highlighter-rouge">false</code> (default: <code class="language-plaintext highlighter-rouge">true</code>).</p>
<p>Spark can be configured to use the following authentication protocols to obtain token (it must match with
Kafka broker configuration):</p>
<ul>
<li><strong>SASL SSL (default)</strong></li>
<li><strong>SSL</strong></li>
<li><strong>SASL PLAINTEXT (for testing)</strong></li>
</ul>
<p>After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly.
Delegation token uses <code class="language-plaintext highlighter-rouge">SCRAM</code> login module for authentication and because of that the appropriate
<code class="language-plaintext highlighter-rouge">spark.kafka.clusters.${cluster}.sasl.token.mechanism</code> (default: <code class="language-plaintext highlighter-rouge">SCRAM-SHA-512</code>) has to be configured. Also, this parameter
must match with Kafka broker configuration.</p>
<p>When delegation token is available on an executor Spark considers the following log in options, in order of preference:</p>
<ul>
<li><strong>JAAS login configuration</strong>, please see example below.</li>
<li><strong>Delegation token</strong>, please see <code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code> parameter for further details.</li>
</ul>
<p>When none of the above applies then unsecure connection assumed.</p>
<h4 id="configuration">Configuration</h4>
<p>Delegation tokens can be obtained from multiple clusters and <code>${cluster}</code> is an arbitrary unique identifier which helps to group different configurations.</p>
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.auth.bootstrap.servers</code></td>
<td>None</td>
<td>
A list of coma separated host/port pairs to use for establishing the initial connection
to the Kafka cluster. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code></td>
<td>.*</td>
<td>
Regular expression to match against the <code>bootstrap.servers</code> config for sources and sinks in the application.
If a server address matches this regex, the delegation token obtained from the respective bootstrap servers will be used when connecting.
If multiple clusters match the address, an exception will be thrown and the query won't be started.
Kafka's secure and unsecure listeners are bound to different ports. When both used the secure listener port has to be part of the regular expression.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.security.protocol</code></td>
<td>SASL_SSL</td>
<td>
Protocol used to communicate with brokers. For further details please see Kafka documentation. Protocol is applied on all the sources and sinks as default where
<code>bootstrap.servers</code> config matches (for further details please see <code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code>),
and can be overridden by setting <code>kafka.security.protocol</code> on the source or sink.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.sasl.kerberos.service.name</code></td>
<td>kafka</td>
<td>
The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.type</code></td>
<td>None</td>
<td>
The file format of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.2.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.location</code></td>
<td>None</td>
<td>
The location of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.password</code></td>
<td>None</td>
<td>
The store password for the trust store file. This is optional and only needed if <code>spark.kafka.clusters.${cluster}.ssl.truststore.location</code> is configured.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.type</code></td>
<td>None</td>
<td>
The file format of the key store file. This is optional for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.2.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.location</code></td>
<td>None</td>
<td>
The location of the key store file. This is optional for client and can be used for two-way authentication for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.password</code></td>
<td>None</td>
<td>
The store password for the key store file. This is optional and only needed if <code>spark.kafka.clusters.${cluster}.ssl.keystore.location</code> is configured.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.key.password</code></td>
<td>None</td>
<td>
The password of the private key in the key store file. This is optional for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.sasl.token.mechanism</code></td>
<td>SCRAM-SHA-512</td>
<td>
SASL mechanism used for client connections with delegation token. Because SCRAM login module used for authentication a compatible mechanism has to be set here.
For further details please see Kafka documentation (<code>sasl.mechanism</code>). Only used to authenticate against Kafka broker with delegation token.
</td>
<td>3.0.0</td>
</tr>
</table>
<h4 id="kafka-specific-configurations-1">Kafka Specific Configurations</h4>
<p>Kafka&#8217;s own configurations can be set with <code class="language-plaintext highlighter-rouge">kafka.</code> prefix, e.g, <code class="language-plaintext highlighter-rouge">--conf spark.kafka.clusters.${cluster}.kafka.retries=1</code>.
For possible Kafka parameters, see <a href="http://kafka.apache.org/documentation.html#adminclientconfigs">Kafka adminclient config docs</a>.</p>
<h4 id="caveats">Caveats</h4>
<ul>
<li>Obtaining delegation token for proxy user is not yet supported (<a href="https://issues.apache.org/jira/browse/KAFKA-6945">KAFKA-6945</a>).</li>
</ul>
<h3 id="jaas-login-configuration">JAAS login configuration</h3>
<p>JAAS login configuration must placed on all nodes where Spark tries to access Kafka cluster.
This provides the possibility to apply any custom authentication logic with a higher cost to maintain.
This can be done several ways. One possibility is to provide additional JVM parameters, such as,</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-submit \
--driver-java-options "-Djava.security.auth.login.config=/path/to/custom_jaas.conf" \
--conf spark.executor.extraJavaOptions=-Djava.security.auth.login.config=/path/to/custom_jaas.conf \
...
</code></pre></div></div>
</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="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/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.4.3"]
},
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>